From faffdd69bce7a8c1c4eb17d09c82ff8fc3a53f2e Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Mon, 8 Apr 2024 19:48:35 +0900
Subject: [PATCH 01/48] Support micrometer context-propagation : Skeleton code
---
reactor3/build.gradle | 1 +
.../reactor3/RequestContextAccessor.java | 55 ++
.../RequestContextPropagationHook.java | 33 ++
.../RequestContextPropagationFluxTest.java | 478 ++++++++++++++++++
.../RequestContextPropagationMonoTest.java | 320 ++++++++++++
spring/boot3-autoconfigure/build.gradle | 1 +
.../spring/ArmeriaConfigurationUtil.java | 5 +
.../armeria/spring/ArmeriaSettings.java | 11 +
.../ArmeriaSettingsConfigurationTest.java | 3 +
.../resources/config/application-settings.yml | 1 +
10 files changed, 908 insertions(+)
create mode 100644 reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
create mode 100644 reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
create mode 100644 reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
create mode 100644 reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
diff --git a/reactor3/build.gradle b/reactor3/build.gradle
index d3eb4fd5551..e2b7194a7e6 100644
--- a/reactor3/build.gradle
+++ b/reactor3/build.gradle
@@ -1,3 +1,4 @@
dependencies {
api libs.reactor.core
+ implementation 'io.micrometer:context-propagation:1.1.1'
}
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
new file mode 100644
index 00000000000..fecf00adabe
--- /dev/null
+++ b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
@@ -0,0 +1,55 @@
+package com.linecorp.armeria.common.reactor3;
+
+import java.util.Objects;
+
+import com.linecorp.armeria.common.RequestContext;
+import com.linecorp.armeria.internal.common.RequestContextUtil;
+
+import io.micrometer.context.ThreadLocalAccessor;
+
+public class RequestContextAccessor implements ThreadLocalAccessor {
+
+ private static final String KEY = "ARMERIA_REQUEST_CONTEXT";
+ private static final RequestContextAccessor instance = createInstance();
+
+ private static RequestContextAccessor createInstance() {
+ return new RequestContextAccessor();
+ }
+
+ public static RequestContextAccessor getInstance() {
+ return instance;
+ }
+
+ private RequestContextAccessor() {
+ }
+
+ @Override
+ public Object key() {
+ return KEY;
+ }
+
+ @Override
+ public RequestContext getValue() {
+ return RequestContextUtil.get();
+ }
+
+ @Override
+ public void setValue(RequestContext value) {
+ RequestContextUtil.getAndSet(value);
+ }
+
+ @Override
+ public void setValue() {
+ // NO Operation.
+ }
+
+ @Override
+ public void restore(RequestContext previousValue) {
+ RequestContextUtil.getAndSet(previousValue);
+ }
+
+ @Override
+ public void restore() {
+ RequestContextUtil.pop();
+ }
+}
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
new file mode 100644
index 00000000000..8e6884b80ca
--- /dev/null
+++ b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
@@ -0,0 +1,33 @@
+package com.linecorp.armeria.common.reactor3;
+
+
+import io.micrometer.context.ContextRegistry;
+import reactor.core.publisher.Hooks;
+
+public class RequestContextPropagationHook {
+
+ private static boolean enabled;
+
+ public static synchronized void enable() {
+ if (enabled) {
+ return;
+ }
+ ContextRegistry.getInstance().registerThreadLocalAccessor(RequestContextAccessor.getInstance());
+ Hooks.enableAutomaticContextPropagation();
+
+ enabled = true;
+ }
+
+ public static boolean isEnable() {
+ return enabled;
+ }
+
+ public static synchronized void disable() {
+ if (!enabled) {
+ return;
+ }
+
+ Hooks.disableAutomaticContextPropagation();
+ enabled = false;
+ }
+}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
new file mode 100644
index 00000000000..c7eff378893
--- /dev/null
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -0,0 +1,478 @@
+/*
+ * Copyright 2020 LINE Corporation
+ *
+ * LINE Corporation licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ * https://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 com.linecorp.armeria.common.reactor3;
+
+import static com.linecorp.armeria.common.reactor3.ContextAwareMonoTest.ctxExists;
+import static com.linecorp.armeria.common.reactor3.ContextAwareMonoTest.newContext;
+import static com.linecorp.armeria.common.reactor3.ContextAwareMonoTest.noopSubscription;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import java.time.Duration;
+import java.util.stream.Stream;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.reactivestreams.Publisher;
+
+import com.linecorp.armeria.client.ClientRequestContext;
+import com.linecorp.armeria.internal.testing.AnticipatedException;
+import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
+
+import reactor.core.Disposable;
+import reactor.core.publisher.ConnectableFlux;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.Mono;
+import reactor.core.scheduler.Schedulers;
+import reactor.test.StepVerifier;
+
+@GenerateNativeImageTrace
+class RequestContextPropagationFluxTest {
+
+ @BeforeAll
+ static void setUp() {
+ RequestContextPropagationHook.enable();
+ }
+
+ @AfterAll
+ static void tearDown() {
+ RequestContextPropagationHook.disable();
+ }
+
+ // No need to be tested.
+ /*
+ @Test
+ void fluxJust() {
+ // FluxJust and FluxEmpty are a scalar type and could be subscribed by multiple requests.
+ // Therefore, Flux.just(...), Flux.empty() and Flux.error(ex) should not return a ContextAwareFlux.
+ final Flux just = Flux.just("foo");
+ final Flux empty = Flux.empty();
+ final Flux error = Flux.error(new IllegalStateException("boom"));
+ assertThat(just).isNotExactlyInstanceOf(ContextAwareMono.class);
+ assertThat(empty).isNotExactlyInstanceOf(ContextAwareMono.class);
+ assertThat(error).isNotExactlyInstanceOf(ContextAwareMono.class);
+ }
+ */
+
+ @Test
+ void fluxError() {
+ final ClientRequestContext ctx = newContext();
+ final Flux
+ * However, please note that you should include {@link Mono#contextWrite(ContextView)} or
+ * {@link Flux#contextWrite(ContextView)} to end of the Reactor codes.
+ * If not, {@link RequestContext} will not be keep during Reactor Operation.
*/
public static RequestContextAccessor getInstance() {
return instance;
@@ -42,39 +64,98 @@ public static RequestContextAccessor getInstance() {
private RequestContextAccessor() {
}
+ /**
+ * The value which obtained through {@link RequestContextAccessor},
+ * will be stored in the {@link Context} under this {@code KEY}.
+ */
@Override
public Object key() {
return KEY;
}
+ /**
+ * {@link ContextSnapshot} will call this method during the execution
+ * of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
+ * as well as during {@link Mono#subscribe()}, {@link Flux#subscribe()},
+ * {@link Subscription#request(long)}, and {@link CoreSubscriber#onSubscribe(Subscription)}.
+ * Following these calls, {@link ContextSnapshot#setThreadLocals()} is
+ * invoked to restore the state of {@link RequestContextStorage}.
+ * Furthermore, at the end of these methods, {@link Scope#close()} is executed
+ * to revert the {@link RequestContextStorage} to its original state.
+ */
@Override
public RequestContext getValue() {
return RequestContextUtil.get();
}
+ /**
+ * {@link ContextSnapshot} will call this method during the execution
+ * of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
+ * as well as during {@link Mono#subscribe()}, {@link Flux#subscribe()},
+ * {@link Subscription#request(long)}, and {@link CoreSubscriber#onSubscribe(Subscription)}.
+ * Following these calls, {@link ContextSnapshot#setThreadLocals()} is
+ * invoked to restore the state of {@link RequestContextStorage}.
+ * Furthermore, at the end of these methods, {@link Scope#close()} is executed
+ * to revert the {@link RequestContextStorage} to its original state.
+ */
@Override
public void setValue(RequestContext value) {
RequestContextUtil.getAndSet(value);
}
+ /**
+ * This method will be called at the start of {@link ContextSnapshot.Scope} and
+ * the end of {@link ContextSnapshot.Scope}. If reactor {@link Context} does not
+ * contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
+ * this method to remove the value from {@link ThreadLocal}.
+ *
+ * Please note that {@link RequestContextUtil#pop()} return {@link AutoCloseable} instance,
+ * but it is not used in `Try with Resources` syntax. this is because {@link ContextSnapshot.Scope}
+ * will handle the {@link AutoCloseable} instance returned by {@link RequestContextUtil#pop()}.
+ */
@Override
+ @SuppressWarnings("MustBeClosedChecker")
public void setValue() {
- // NO Operation.
- // This method is called when DefaultScope is closed and no previous value existed in
- // ThreadLocal State.
+ RequestContextUtil.pop();
}
+ /**
+ * {@link ContextSnapshot} will call this method during the execution
+ * of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
+ * as well as during {@link Mono#subscribe()}, {@link Flux#subscribe()},
+ * {@link Subscription#request(long)}, and {@link CoreSubscriber#onSubscribe(Subscription)}.
+ * Following these calls, {@link ContextSnapshot#setThreadLocals()} is
+ * invoked to restore the state of {@link RequestContextStorage}.
+ * Furthermore, at the end of these methods, {@link Scope#close()} is executed
+ * to revert the {@link RequestContextStorage} to its original state.
+ */
@Override
public void restore(RequestContext previousValue) {
RequestContextUtil.getAndSet(previousValue);
}
- /*
+ /**
+ * This method will be called at the start of {@link ContextSnapshot.Scope} and
+ * the end of {@link ContextSnapshot.Scope}. If reactor {@link Context} does not
+ * contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
+ * this method to remove the value from {@link ThreadLocal}.
+ *
+ * Please note that {@link RequestContextUtil#pop()} return {@link AutoCloseable} instance,
+ * but it is not used in `Try with Resources` syntax. this is because {@link ContextSnapshot.Scope}
+ * will handle the {@link AutoCloseable} instance returned by {@link RequestContextUtil#pop()}.
+ */
@Override
+ @SuppressWarnings("MustBeClosedChecker")
public void restore() {
- // Use super.restore() instead of implementing on child class.
- // This method is called when DefaultContextSnapshot.DefaultContextScope
- // is closed and no previous value existed in ThreadLocal State.
+ RequestContextUtil.pop();
}
+
+ /**
+ * This is a utility method to get a Reactor {@link Context} that contains {@link RequestContext}.
+ * You can use this method for both {@link Mono#contextWrite(ContextView)}
+ * and {@link Flux#contextWrite(ContextView)}.
*/
+ public Context getReactorContext(RequestContext ctx) {
+ return Context.of(KEY, ctx);
+ }
}
From 1c3cda09ad10fb9720269b3822b90fad9800a36b Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 10 Apr 2024 16:47:35 +0900
Subject: [PATCH 04/48] Add java docs and modify unit test
---
.../reactor3/RequestContextAccessor.java | 2 +-
.../RequestContextPropagationHook.java | 38 ++++--
.../RequestContextPropagationFluxTest.java | 128 +++++-------------
.../RequestContextPropagationMonoTest.java | 108 ++++-----------
4 files changed, 88 insertions(+), 188 deletions(-)
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
index 8ced5f596d0..c7346b38319 100644
--- a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
+++ b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
@@ -34,7 +34,7 @@
/**
* This class works with the
*
- * Context-propagation library and keep the {@link RequestContext} during
+ * Context-propagation library and keep the {@link RequestContext} during
* Reactor operations.
*/
public final class RequestContextAccessor implements ThreadLocalAccessor {
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
index 4d3d193f60c..6351aa9396f 100644
--- a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
+++ b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
@@ -13,44 +13,56 @@
* License for the specific language governing permissions and limitations
* under the License.
*/
+
package com.linecorp.armeria.common.reactor3;
+import com.linecorp.armeria.common.RequestContext;
+
import io.micrometer.context.ContextRegistry;
+import reactor.core.publisher.Flux;
import reactor.core.publisher.Hooks;
+import reactor.core.publisher.Mono;
/**
- * TBD.
+ * Utility class to keep {@link RequestContext} during
+ * Reactor operations
+ * with
+ * Context-propagation.
*/
public final class RequestContextPropagationHook {
- private static boolean enabled;
-
- private RequestContextPropagationHook() {}
+ private static volatile boolean enabled;
/**
- * TBD.
+ * Enable
+ * Context-propagation to keep {@link RequestContext} during
+ * Reactor operations.
+ *
+ * Please note that enable {@link RequestContextPropagationHook} at the
+ * start of the application. otherwise, {@link RequestContext} may not be keep.
*/
public static synchronized void enable() {
if (enabled) {
return;
}
- ContextRegistry
- .getInstance()
- .registerThreadLocalAccessor(RequestContextAccessor.getInstance());
- Hooks.enableAutomaticContextPropagation();
+ final RequestContextAccessor accessor = RequestContextAccessor.getInstance();
+ ContextRegistry.getInstance()
+ .registerThreadLocalAccessor(accessor);
+ Hooks.enableAutomaticContextPropagation();
enabled = true;
}
/**
- * TBD.
+ * It returns whether the {@link RequestContextPropagationHook} is enabled.
*/
- public static boolean isEnable() {
+ public static boolean isEnabled() {
return enabled;
}
/**
- * TBD.
+ * It disable {@link RequestContextPropagationHook}. {@link RequestContext}
+ * will not be keep during both {@link Mono} and {@link Flux} Operations.
*/
public static synchronized void disable() {
if (!enabled) {
@@ -60,4 +72,6 @@ public static synchronized void disable() {
Hooks.disableAutomaticContextPropagation();
enabled = false;
}
+
+ private RequestContextPropagationHook() {}
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index c7eff378893..11c82dc53b5 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -39,6 +39,8 @@
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
+import reactor.test.StepVerifierOptions;
+import reactor.util.context.Context;
@GenerateNativeImageTrace
class RequestContextPropagationFluxTest {
@@ -53,35 +55,19 @@ static void tearDown() {
RequestContextPropagationHook.disable();
}
- // No need to be tested.
- /*
- @Test
- void fluxJust() {
- // FluxJust and FluxEmpty are a scalar type and could be subscribed by multiple requests.
- // Therefore, Flux.just(...), Flux.empty() and Flux.error(ex) should not return a ContextAwareFlux.
- final Flux just = Flux.just("foo");
- final Flux empty = Flux.empty();
- final Flux error = Flux.error(new IllegalStateException("boom"));
- assertThat(just).isNotExactlyInstanceOf(ContextAwareMono.class);
- assertThat(empty).isNotExactlyInstanceOf(ContextAwareMono.class);
- assertThat(error).isNotExactlyInstanceOf(ContextAwareMono.class);
- }
- */
-
@Test
void fluxError() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
- flux = addCallbacks(Flux.error(() -> {
- assertThat(ctxExists(ctx)).isTrue();
- return new AnticipatedException();
- }).publishOn(Schedulers.single()), ctx);
+ flux = addCallbacks(Flux.error(() -> {
+ // This is called twice. after publishOn() and verifyErrorMatches()
+ // After publishOn(), ctxExists(ctx) should be false.
+ // On the other hand, it should be True due to ContextPropagation.
+ return new AnticipatedException();
+ }).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
}
@@ -91,9 +77,6 @@ void fluxFromPublisher() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.from(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.onSubscribe(noopSubscription());
@@ -101,7 +84,7 @@ void fluxFromPublisher() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -112,16 +95,13 @@ void fluxCreate() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.create(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -132,15 +112,12 @@ void fluxCreate_error() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.create(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.error(new AnticipatedException());
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
}
@@ -150,9 +127,6 @@ void fluxConcat() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.concat(Mono.fromSupplier(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
@@ -161,7 +135,7 @@ void fluxConcat() {
return "bar";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.expectNextMatches(s -> ctxExists(ctx) && "bar".equals(s))
@@ -173,15 +147,12 @@ void fluxDefer() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.defer(() -> {
assertThat(ctxExists(ctx)).isTrue();
return Flux.just("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -192,15 +163,12 @@ void fluxFromStream() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.fromStream(() -> {
assertThat(ctxExists(ctx)).isTrue();
return Stream.of("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -211,15 +179,12 @@ void fluxCombineLatest() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.combineLatest(Mono.just("foo"), Mono.just("bar"), (a, b) -> {
assertThat(ctxExists(ctx)).isTrue();
return a;
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -230,16 +195,13 @@ void fluxGenerate() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.generate(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -250,9 +212,6 @@ void fluxMerge() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.mergeSequential(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.onSubscribe(noopSubscription());
@@ -265,7 +224,7 @@ void fluxMerge() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.expectNextMatches(s -> ctxExists(ctx) && "bar".equals(s))
@@ -277,16 +236,13 @@ void fluxPush() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.push(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -297,9 +253,6 @@ void fluxSwitchOnNext() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.switchOnNext(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.onSubscribe(noopSubscription());
@@ -312,7 +265,7 @@ void fluxSwitchOnNext() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -323,15 +276,12 @@ void fluxZip() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.zip(Mono.just("foo"), Mono.just("bar"), (foo, bar) -> {
assertThat(ctxExists(ctx)).isTrue();
return foo;
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -342,15 +292,12 @@ void fluxInterval() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.interval(Duration.ofMillis(100)).take(2).concatMap(a -> {
assertThat(ctxExists(ctx)).isTrue();
return Mono.just("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
@@ -362,9 +309,6 @@ void fluxConcatDelayError() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.concatDelayError(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.onSubscribe(noopSubscription());
@@ -376,7 +320,7 @@ void fluxConcatDelayError() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.expectNextMatches(s -> ctxExists(ctx) && "bar".equals(s))
@@ -388,9 +332,6 @@ void fluxTransform() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = addCallbacks(Flux.just("foo").transform(fooFlux -> s -> {
assertThat(ctxExists(ctx)).isTrue();
s.onSubscribe(noopSubscription());
@@ -398,7 +339,7 @@ void fluxTransform() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -409,14 +350,11 @@ void connectableFlux() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
final ConnectableFlux connectableFlux = Flux.just("foo").publish();
flux = addCallbacks(connectableFlux.autoConnect(2).publishOn(Schedulers.single()), ctx);
flux.subscribe().dispose();
- StepVerifier.create(flux)
+ StepVerifier.create(flux, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -427,9 +365,6 @@ void connectableFlux_dispose() throws InterruptedException {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
final ConnectableFlux connectableFlux = Flux.just("foo").publish();
flux = addCallbacks(connectableFlux.autoConnect(2, disposable -> {
assertThat(ctxExists(ctx)).isTrue();
@@ -449,16 +384,13 @@ void subscriberContextIsNotMissing() {
final ClientRequestContext ctx = newContext();
final Flux flux;
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
flux = Flux.deferContextual(reactorCtx -> {
assertThat((String) reactorCtx.get("foo")).isEqualTo("bar");
return Flux.just("baz");
});
final Flux flux1 = flux.contextWrite(reactorCtx -> reactorCtx.put("foo", "bar"));
- StepVerifier.create(flux1)
+ StepVerifier.create(flux1, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "baz".equals(s))
.verifyComplete();
@@ -472,7 +404,13 @@ private static Flux addCallbacks(Flux flux, ClientRequestContext ctx)
.doOnComplete(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
- .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
+ .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue())
+ .contextWrite(Context.of(RequestContextAccessor.getInstance().key(), ctx));
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
}
+
+ private static StepVerifierOptions initialReactorContext(ClientRequestContext ctx) {
+ final Context reactorCtx = Context.of(RequestContextAccessor.getInstance().key(), ctx);
+ return StepVerifierOptions.create().withInitialContext(reactorCtx);
+ }
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 9cb90ec5484..a084761060c 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -35,6 +35,8 @@
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
+import reactor.test.StepVerifierOptions;
+import reactor.util.context.Context;
import reactor.util.function.Tuple2;
@GenerateNativeImageTrace
@@ -50,35 +52,16 @@ static void tearDown() {
RequestContextPropagationHook.disable();
}
- // No need to be tested.
- /*
- @Test
- void monoJust() {
- // MonoJust is a scalar type and could be subscribed by multiple requests.
- // Therefore, Mono.just(...), Mono.empty() and Mono.error(ex) should not return a ContextAwareMono.
- final Mono mono = Mono.just("foo");
- final Mono empty = Mono.empty();
- final Mono error = Mono.error(new IllegalStateException("boom"));
- assertThat(mono).isNotExactlyInstanceOf(ContextAwareMono.class);
- assertThat(empty).isNotExactlyInstanceOf(ContextAwareMono.class);
- assertThat(error).isNotExactlyInstanceOf(ContextAwareMono.class);
- }
- */
-
@Test
void monoCreate_success() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.success("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -88,16 +71,12 @@ void monoCreate_success() {
void monoCreate_error() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.error(new AnticipatedException());
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
}
@@ -106,16 +85,12 @@ void monoCreate_error() {
void monoCreate_currentContext() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.success("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -125,16 +100,12 @@ void monoCreate_currentContext() {
void monoDefer() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.defer(() -> Mono.fromSupplier(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -144,10 +115,6 @@ void monoDefer() {
void monoFromPublisher() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.from(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.onSubscribe(noopSubscription());
@@ -155,7 +122,7 @@ void monoFromPublisher() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -165,16 +132,12 @@ void monoFromPublisher() {
void monoError() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.error(() -> {
assertThat(ctxExists(ctx)).isTrue();
return new AnticipatedException();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
}
@@ -183,17 +146,13 @@ void monoError() {
void monoFirst() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.firstWithSignal(Mono.delay(Duration.ofMillis(1000)).then(Mono.just("bar")),
Mono.fromCallable(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -203,16 +162,11 @@ void monoFirst() {
void monoFromFuture() {
final CompletableFuture future = new CompletableFuture<>();
future.complete("foo");
-
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.fromFuture(future).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -224,16 +178,12 @@ void monoDelay() {
future.complete("foo");
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = addCallbacks(Mono.delay(Duration.ofMillis(100)).then(Mono.fromCallable(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono)
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
.verifyComplete();
@@ -245,19 +195,15 @@ void monoZip() {
future.complete("foo");
final ClientRequestContext ctx = newContext();
final Mono> mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
- mono = addCallbacks(Mono.zip(Mono.fromSupplier(() -> {
- assertThat(ctxExists(ctx)).isTrue();
- return "foo";
- }), Mono.fromSupplier(() -> {
- assertThat(ctxExists(ctx)).isTrue();
- return "bar";
- })).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
+ mono = addCallbacks(Mono.zip(Mono.fromSupplier(() -> {
+ assertThat(ctxExists(ctx)).isTrue();
+ return "foo";
+ }), Mono.fromSupplier(() -> {
+ assertThat(ctxExists(ctx)).isTrue();
+ return "bar";
+ })).publishOn(Schedulers.single()), ctx);
+
+ StepVerifier.create(mono, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(t -> ctxExists(ctx) &&
"foo".equals(t.getT1()) && "bar".equals(t.getT2()))
@@ -268,17 +214,13 @@ void monoZip() {
void subscriberContextIsNotMissing() {
final ClientRequestContext ctx = newContext();
final Mono mono;
-
- final RequestContextAccessor instance = RequestContextAccessor.getInstance();
- instance.setValue(ctx);
-
mono = Mono.deferContextual(Mono::just).handle((reactorCtx, sink) -> {
assertThat((String) reactorCtx.get("foo")).isEqualTo("bar");
sink.next("baz");
});
final Mono mono1 = mono.contextWrite(reactorCtx -> reactorCtx.put("foo", "bar"));
- StepVerifier.create(mono1)
+ StepVerifier.create(mono1, initialReactorContext(ctx))
.expectSubscriptionMatches(s -> ctxExists(ctx))
.expectNextMatches(s -> ctxExists(ctx) && "baz".equals(s))
.verifyComplete();
@@ -311,7 +253,13 @@ private static Mono addCallbacks(Mono mono, ClientRequestContext ctx)
.doOnSuccess(t -> assertThat(ctxExists(ctx)).isTrue())
.doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
- .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
+ .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue())
+ .contextWrite(Context.of(RequestContextAccessor.getInstance().key(), ctx));
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
}
+
+ private static StepVerifierOptions initialReactorContext(ClientRequestContext ctx) {
+ final Context reactorCtx = Context.of(RequestContextAccessor.getInstance().key(), ctx);
+ return StepVerifierOptions.create().withInitialContext(reactorCtx);
+ }
}
From 8030e8fbea4c4f7d67efe099463d9dc20972e127 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 10 Apr 2024 16:50:39 +0900
Subject: [PATCH 05/48] Apply comment
---
.../armeria/spring/ArmeriaSettingsConfigurationTest.java | 3 +--
.../src/test/resources/config/application-settings.yml | 1 -
2 files changed, 1 insertion(+), 3 deletions(-)
diff --git a/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java b/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java
index e541145bfeb..1e32aebf4f0 100644
--- a/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java
+++ b/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java
@@ -70,7 +70,6 @@ void buildServerBasedOnProperties() {
assertThat(defaultVirtualHost.requestTimeoutMillis()).isEqualTo(8000);
assertThat(defaultVirtualHost.maxRequestLength()).isEqualTo(0);
assertThat(defaultVirtualHost.verboseResponses()).isTrue();
-
- assertThat(RequestContextPropagationHook.isEnable()).isTrue();
+ assertThat(RequestContextPropagationHook.isEnabled()).isFalse();
}
}
diff --git a/spring/boot3-autoconfigure/src/test/resources/config/application-settings.yml b/spring/boot3-autoconfigure/src/test/resources/config/application-settings.yml
index cba389e3951..8db7618d241 100644
--- a/spring/boot3-autoconfigure/src/test/resources/config/application-settings.yml
+++ b/spring/boot3-autoconfigure/src/test/resources/config/application-settings.yml
@@ -17,4 +17,3 @@ armeria:
request-timeout: 8000
max-request-length: 0
verbose-responses: true
- enable-context-propagation: true
From 46cd890a1fbae2c9dc8fa0edc992281b5024883d Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 14:47:37 +0900
Subject: [PATCH 06/48] remove ContextPropagation from boot3-autoconfigure
---
spring/boot3-autoconfigure/build.gradle | 1 -
.../internal/spring/ArmeriaConfigurationUtil.java | 7 -------
.../com/linecorp/armeria/spring/ArmeriaSettings.java | 11 -----------
.../spring/ArmeriaSettingsConfigurationTest.java | 2 --
4 files changed, 21 deletions(-)
diff --git a/spring/boot3-autoconfigure/build.gradle b/spring/boot3-autoconfigure/build.gradle
index 76f7b9f3f0e..1fe31ad461c 100644
--- a/spring/boot3-autoconfigure/build.gradle
+++ b/spring/boot3-autoconfigure/build.gradle
@@ -2,7 +2,6 @@ dependencies {
// To let a user choose between thrift and thrift0.9.
compileOnly project(':thrift0.18')
implementation project(':logback')
- implementation project(':reactor3')
// TODO(anuraaga): Consider removing these since this module does not have related functionality.
optionalApi libs.micrometer.prometheus
diff --git a/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/internal/spring/ArmeriaConfigurationUtil.java b/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/internal/spring/ArmeriaConfigurationUtil.java
index 159d4c4c081..580e3702597 100644
--- a/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/internal/spring/ArmeriaConfigurationUtil.java
+++ b/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/internal/spring/ArmeriaConfigurationUtil.java
@@ -62,7 +62,6 @@
import com.linecorp.armeria.common.SessionProtocol;
import com.linecorp.armeria.common.annotation.Nullable;
import com.linecorp.armeria.common.metric.MeterIdPrefixFunction;
-import com.linecorp.armeria.common.reactor3.RequestContextPropagationHook;
import com.linecorp.armeria.server.HttpService;
import com.linecorp.armeria.server.ServerBuilder;
import com.linecorp.armeria.server.ServerErrorHandler;
@@ -206,13 +205,7 @@ public static void configureServerWithArmeriaSettings(
if (settings.isEnableAutoInjection()) {
server.dependencyInjector(SpringDependencyInjector.of(beanFactory), false);
}
-
- if (settings.isEnableContextPropagation()) {
- RequestContextPropagationHook.enable();
- }
-
serverErrorHandlers.forEach(server::errorHandler);
-
}
private static void configureInternalService(ServerBuilder server, InternalServiceId serviceId,
diff --git a/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/spring/ArmeriaSettings.java b/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/spring/ArmeriaSettings.java
index ea35a02bcfb..d3226fab384 100644
--- a/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/spring/ArmeriaSettings.java
+++ b/spring/boot3-autoconfigure/src/main/java/com/linecorp/armeria/spring/ArmeriaSettings.java
@@ -629,9 +629,6 @@ public void setFormat(@Nullable String format) {
*/
private boolean enableAutoInjection;
- private boolean enableContextPropagation;
-
-
/**
* Returns the {@link Port}s of the {@link Server}.
*/
@@ -1125,12 +1122,4 @@ public boolean isEnableAutoInjection() {
public void setEnableAutoInjection(boolean enableAutoInjection) {
this.enableAutoInjection = enableAutoInjection;
}
-
- public boolean isEnableContextPropagation() {
- return enableContextPropagation;
- }
-
- public void setEnableContextPropagation(boolean enableContextPropagation) {
- this.enableContextPropagation = enableContextPropagation;
- }
}
diff --git a/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java b/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java
index 1e32aebf4f0..18e961d8d59 100644
--- a/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java
+++ b/spring/boot3-autoconfigure/src/test/java/com/linecorp/armeria/spring/ArmeriaSettingsConfigurationTest.java
@@ -27,7 +27,6 @@
import org.springframework.test.context.ActiveProfiles;
import com.linecorp.armeria.common.annotation.Nullable;
-import com.linecorp.armeria.common.reactor3.RequestContextPropagationHook;
import com.linecorp.armeria.server.Server;
import com.linecorp.armeria.server.ServerConfig;
import com.linecorp.armeria.server.VirtualHost;
@@ -70,6 +69,5 @@ void buildServerBasedOnProperties() {
assertThat(defaultVirtualHost.requestTimeoutMillis()).isEqualTo(8000);
assertThat(defaultVirtualHost.maxRequestLength()).isEqualTo(0);
assertThat(defaultVirtualHost.verboseResponses()).isTrue();
- assertThat(RequestContextPropagationHook.isEnabled()).isFalse();
}
}
From 67a4f1413f2d61a9404dd437cf48ca2788f12150 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 16:42:43 +0900
Subject: [PATCH 07/48] Move RequestContextAccessor to core module.
---
core/build.gradle | 1 +
.../common}/RequestContextAccessor.java | 79 ++++------
.../io.micrometer.context.ThreadLocalAccessor | 1 +
.../common/RequestContextAccessorTest.java | 144 ++++++++++++++++++
.../RequestContextPropagationHook.java | 5 -
.../RequestContextPropagationFluxTest.java | 5 +-
.../RequestContextPropagationMonoTest.java | 5 +-
7 files changed, 183 insertions(+), 57 deletions(-)
rename {reactor3/src/main/java/com/linecorp/armeria/common/reactor3 => core/src/main/java/com/linecorp/armeria/common}/RequestContextAccessor.java (65%)
create mode 100644 core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
create mode 100644 core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
diff --git a/core/build.gradle b/core/build.gradle
index f4811c674ab..b0825a97d83 100644
--- a/core/build.gradle
+++ b/core/build.gradle
@@ -108,6 +108,7 @@ dependencies {
optionalApi libs.micrometer.prometheus
optionalApi libs.dropwizard.metrics.core
optionalApi libs.prometheus
+ implementation 'io.micrometer:context-propagation:1.1.1'
// Netty
api libs.netty.transport
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
similarity index 65%
rename from reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
rename to core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index c7346b38319..b3442a70234 100644
--- a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -13,71 +13,63 @@
* License for the specific language governing permissions and limitations
* under the License.
*/
-package com.linecorp.armeria.common.reactor3;
+package com.linecorp.armeria.common;
import org.reactivestreams.Subscription;
-import com.linecorp.armeria.common.RequestContext;
-import com.linecorp.armeria.common.RequestContextStorage;
import com.linecorp.armeria.internal.common.RequestContextUtil;
import io.micrometer.context.ContextRegistry;
import io.micrometer.context.ContextSnapshot;
import io.micrometer.context.ContextSnapshot.Scope;
import io.micrometer.context.ThreadLocalAccessor;
-import reactor.core.CoreSubscriber;
-import reactor.core.publisher.Flux;
-import reactor.core.publisher.Mono;
-import reactor.util.context.Context;
-import reactor.util.context.ContextView;
/**
* This class works with the
*
* Context-propagation library and keep the {@link RequestContext} during
* Reactor operations.
+ *
+ * Get the {@link RequestContextAccessor} to register it to the {@link ContextRegistry}.
+ * Then, {@link ContextRegistry} will use {@link RequestContextAccessor} to
+ * propagate context during the
+ * Reactor operations
+ * so that you can get the context using {@link RequestContext#current()}.
+ *
+ * However, please note that you should include Mono#contextWrite(ContextView) or
+ * Flux#contextWrite(ContextView) to end of the Reactor codes.
+ * If not, {@link RequestContext} will not be keep during Reactor Operation.
*/
public final class RequestContextAccessor implements ThreadLocalAccessor {
- private static final String KEY = "ARMERIA_REQUEST_CONTEXT";
- private static final RequestContextAccessor instance = createInstance();
-
- private static RequestContextAccessor createInstance() {
- return new RequestContextAccessor();
- }
+ private static final String KEY = RequestContextAccessor.class.getName();
/**
- * Get the {@link RequestContextAccessor} to register it to the {@link ContextRegistry}.
- * Then, {@link ContextRegistry} will use {@link RequestContextAccessor} to
- * propagate context during the
- * Reactor operations
- * so that you can get the context using {@link RequestContext#current()}.
- *
- * However, please note that you should include {@link Mono#contextWrite(ContextView)} or
- * {@link Flux#contextWrite(ContextView)} to end of the Reactor codes.
- * If not, {@link RequestContext} will not be keep during Reactor Operation.
+ * The value which obtained through {@link RequestContextAccessor},
+ * will be stored in the Context under this {@code KEY}.
+ * This method will be called by {@link ContextSnapshot} internally.
*/
- public static RequestContextAccessor getInstance() {
- return instance;
- }
-
- private RequestContextAccessor() {
+ @Override
+ public Object key() {
+ return KEY;
}
/**
* The value which obtained through {@link RequestContextAccessor},
- * will be stored in the {@link Context} under this {@code KEY}.
+ * will be stored in the Context under this {@code KEY}.
+ * User can use this method to register {@link RequestContext} to
+ * Reactor Context.
*/
- @Override
- public Object key() {
+
+ public static String accessorKey() {
return KEY;
}
/**
* {@link ContextSnapshot} will call this method during the execution
* of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
- * as well as during {@link Mono#subscribe()}, {@link Flux#subscribe()},
- * {@link Subscription#request(long)}, and {@link CoreSubscriber#onSubscribe(Subscription)}.
+ * as well as during Mono#subscribe(), Flux#subscribe(),
+ * {@link Subscription#request(long)}, and CoreSubscriber#onSubscribe(Subscription).
* Following these calls, {@link ContextSnapshot#setThreadLocals()} is
* invoked to restore the state of {@link RequestContextStorage}.
* Furthermore, at the end of these methods, {@link Scope#close()} is executed
@@ -91,8 +83,8 @@ public RequestContext getValue() {
/**
* {@link ContextSnapshot} will call this method during the execution
* of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
- * as well as during {@link Mono#subscribe()}, {@link Flux#subscribe()},
- * {@link Subscription#request(long)}, and {@link CoreSubscriber#onSubscribe(Subscription)}.
+ * as well as during Mono#subscribe(), Flux#subscribe(),
+ * {@link Subscription#request(long)}, and CoreSubscriber#onSubscribe(Subscription).
* Following these calls, {@link ContextSnapshot#setThreadLocals()} is
* invoked to restore the state of {@link RequestContextStorage}.
* Furthermore, at the end of these methods, {@link Scope#close()} is executed
@@ -105,7 +97,7 @@ public void setValue(RequestContext value) {
/**
* This method will be called at the start of {@link ContextSnapshot.Scope} and
- * the end of {@link ContextSnapshot.Scope}. If reactor {@link Context} does not
+ * the end of {@link ContextSnapshot.Scope}. If reactor Context does not
* contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
* this method to remove the value from {@link ThreadLocal}.
*
@@ -122,8 +114,8 @@ public void setValue() {
/**
* {@link ContextSnapshot} will call this method during the execution
* of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
- * as well as during {@link Mono#subscribe()}, {@link Flux#subscribe()},
- * {@link Subscription#request(long)}, and {@link CoreSubscriber#onSubscribe(Subscription)}.
+ * as well as during Mono#subscribe(), Flux#subscribe(),
+ * {@link Subscription#request(long)}, and CoreSubscriber#onSubscribe(Subscription).
* Following these calls, {@link ContextSnapshot#setThreadLocals()} is
* invoked to restore the state of {@link RequestContextStorage}.
* Furthermore, at the end of these methods, {@link Scope#close()} is executed
@@ -136,7 +128,7 @@ public void restore(RequestContext previousValue) {
/**
* This method will be called at the start of {@link ContextSnapshot.Scope} and
- * the end of {@link ContextSnapshot.Scope}. If reactor {@link Context} does not
+ * the end of {@link ContextSnapshot.Scope}. If reactor Context does not
* contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
* this method to remove the value from {@link ThreadLocal}.
*
@@ -149,13 +141,4 @@ public void restore(RequestContext previousValue) {
public void restore() {
RequestContextUtil.pop();
}
-
- /**
- * This is a utility method to get a Reactor {@link Context} that contains {@link RequestContext}.
- * You can use this method for both {@link Mono#contextWrite(ContextView)}
- * and {@link Flux#contextWrite(ContextView)}.
- */
- public Context getReactorContext(RequestContext ctx) {
- return Context.of(KEY, ctx);
- }
}
diff --git a/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor b/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
new file mode 100644
index 00000000000..408c5a5ca30
--- /dev/null
+++ b/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
@@ -0,0 +1 @@
+com.linecorp.armeria.common.RequestContextAccessor
diff --git a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
new file mode 100644
index 00000000000..57453bd276e
--- /dev/null
+++ b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
@@ -0,0 +1,144 @@
+/*
+ * Copyright 2019 LINE Corporation
+ *
+ * LINE Corporation licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ * https://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.
+ */
+/*
+ * Copyright 2012 The Netty Project
+ *
+ * The Netty Project licenses this file to you under the Apache License,
+ * version 2.0 (the "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at:
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations
+ * under the License.
+ */
+package com.linecorp.armeria.common;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import com.linecorp.armeria.client.ClientRequestContext;
+
+import io.micrometer.context.ContextRegistry;
+import io.micrometer.context.ThreadLocalAccessor;
+
+class RequestContextAccessorTest {
+
+ @Test
+ void should_be_loaded_by_SPI() {
+ final ContextRegistry ctxRegistry = ContextRegistry.getInstance();
+ final List> threadLocalAccessors = ctxRegistry.getThreadLocalAccessors();
+
+ assertThat(threadLocalAccessors.size()).isGreaterThan(1);
+ assertThat(threadLocalAccessors).hasAtLeastOneElementOfType(RequestContextAccessor.class);
+ }
+
+ @Test
+ void should_return_expected_key() {
+ // Given
+ final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final String expectedValue = RequestContextAccessor.class.getName();
+
+ // When
+ final Object result = reqCtxAccessor.key();
+
+ // Then
+ assertThat(result).isEqualTo(expectedValue);
+ }
+
+ @Test
+ void should_success_set() {
+ // Given
+ final ClientRequestContext ctx = newContext();
+ final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+
+ // When
+ reqCtxAccessor.setValue(ctx);
+
+ // Then
+ final RequestContext currentCtx = RequestContext.current();
+ assertThat(currentCtx).isEqualTo(ctx);
+ }
+
+ @Test
+ void should_throw_NPE_when_set_null() {
+ // Given
+ final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+
+ // When + Then
+ Assertions.assertThatThrownBy(() -> reqCtxAccessor.setValue(null))
+ .isInstanceOf(NullPointerException.class);
+ }
+
+ @Test
+ void should_be_null_when_setValue() {
+ // Given
+ final ClientRequestContext ctx = newContext();
+ final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ reqCtxAccessor.setValue(ctx);
+
+ // When
+ reqCtxAccessor.setValue();
+
+ // Then
+ final RequestContext reqCtx = RequestContext.currentOrNull();
+ assertThat(reqCtx).isNull();
+ }
+
+ @Test
+ void should_be_restore_original_state_when_restore() {
+ // Given
+ final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final ClientRequestContext previousCtx = newContext();
+ final ClientRequestContext currentCtx = newContext();
+ reqCtxAccessor.setValue(currentCtx);
+
+ // When
+ reqCtxAccessor.restore(previousCtx);
+
+ // Then
+ final RequestContext reqCtx = RequestContext.currentOrNull();
+ assertThat(reqCtx).isNotNull();
+ assertThat(reqCtx).isEqualTo(previousCtx);
+ }
+
+ @Test
+ void should_be_null_when_restore() {
+ // Given
+ final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final ClientRequestContext currentCtx = newContext();
+ reqCtxAccessor.setValue(currentCtx);
+
+ // When
+ reqCtxAccessor.restore();
+
+ // Then
+ final RequestContext reqCtx = RequestContext.currentOrNull();
+ assertThat(reqCtx).isNull();
+ }
+
+ static ClientRequestContext newContext() {
+ return ClientRequestContext.builder(HttpRequest.of(HttpMethod.GET, "/"))
+ .build();
+ }
+}
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
index 6351aa9396f..0e79cd96331 100644
--- a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
+++ b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
@@ -18,7 +18,6 @@
import com.linecorp.armeria.common.RequestContext;
-import io.micrometer.context.ContextRegistry;
import reactor.core.publisher.Flux;
import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
@@ -45,10 +44,6 @@ public static synchronized void enable() {
if (enabled) {
return;
}
-
- final RequestContextAccessor accessor = RequestContextAccessor.getInstance();
- ContextRegistry.getInstance()
- .registerThreadLocalAccessor(accessor);
Hooks.enableAutomaticContextPropagation();
enabled = true;
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 11c82dc53b5..c6d3fdae7f6 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -30,6 +30,7 @@
import org.reactivestreams.Publisher;
import com.linecorp.armeria.client.ClientRequestContext;
+import com.linecorp.armeria.common.RequestContextAccessor;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
@@ -405,12 +406,12 @@ private static Flux addCallbacks(Flux flux, ClientRequestContext ctx)
.doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
.doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue())
- .contextWrite(Context.of(RequestContextAccessor.getInstance().key(), ctx));
+ .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
}
private static StepVerifierOptions initialReactorContext(ClientRequestContext ctx) {
- final Context reactorCtx = Context.of(RequestContextAccessor.getInstance().key(), ctx);
+ final Context reactorCtx = Context.of(RequestContextAccessor.accessorKey(), ctx);
return StepVerifierOptions.create().withInitialContext(reactorCtx);
}
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index a084761060c..e72377e1a54 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -29,6 +29,7 @@
import com.linecorp.armeria.common.HttpMethod;
import com.linecorp.armeria.common.HttpRequest;
import com.linecorp.armeria.common.RequestContext;
+import com.linecorp.armeria.common.RequestContextAccessor;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
@@ -254,12 +255,12 @@ private static Mono addCallbacks(Mono mono, ClientRequestContext ctx)
.doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
.doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue())
- .contextWrite(Context.of(RequestContextAccessor.getInstance().key(), ctx));
+ .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
}
private static StepVerifierOptions initialReactorContext(ClientRequestContext ctx) {
- final Context reactorCtx = Context.of(RequestContextAccessor.getInstance().key(), ctx);
+ final Context reactorCtx = Context.of(RequestContextAccessor.accessorKey(), ctx);
return StepVerifierOptions.create().withInitialContext(reactorCtx);
}
}
From ae022ad6ebe88bcda1258b47a64846500bb1aeab Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 16:44:36 +0900
Subject: [PATCH 08/48] remove useless dependency
---
reactor3/build.gradle | 1 -
1 file changed, 1 deletion(-)
diff --git a/reactor3/build.gradle b/reactor3/build.gradle
index e2b7194a7e6..d3eb4fd5551 100644
--- a/reactor3/build.gradle
+++ b/reactor3/build.gradle
@@ -1,4 +1,3 @@
dependencies {
api libs.reactor.core
- implementation 'io.micrometer:context-propagation:1.1.1'
}
From c633eaa4dbb6682e291eb041bf7c8e5f620e901f Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 16:57:04 +0900
Subject: [PATCH 09/48] add dependencies
---
reactor3/build.gradle | 1 +
1 file changed, 1 insertion(+)
diff --git a/reactor3/build.gradle b/reactor3/build.gradle
index d3eb4fd5551..e2b7194a7e6 100644
--- a/reactor3/build.gradle
+++ b/reactor3/build.gradle
@@ -1,3 +1,4 @@
dependencies {
api libs.reactor.core
+ implementation 'io.micrometer:context-propagation:1.1.1'
}
From 8710f1da27666d187be6d052ed1474e3fd960e9a Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 17:07:49 +0900
Subject: [PATCH 10/48] resolve java doc error
---
.../com/linecorp/armeria/common/RequestContextAccessor.java | 4 ----
.../common/reactor3/RequestContextPropagationHook.java | 1 -
2 files changed, 5 deletions(-)
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index b3442a70234..2a040498b56 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -29,13 +29,11 @@
*
* Context-propagation library and keep the {@link RequestContext} during
* Reactor operations.
- *
* Get the {@link RequestContextAccessor} to register it to the {@link ContextRegistry}.
* Then, {@link ContextRegistry} will use {@link RequestContextAccessor} to
* propagate context during the
* Reactor operations
* so that you can get the context using {@link RequestContext#current()}.
- *
* However, please note that you should include Mono#contextWrite(ContextView) or
* Flux#contextWrite(ContextView) to end of the Reactor codes.
* If not, {@link RequestContext} will not be keep during Reactor Operation.
@@ -100,7 +98,6 @@ public void setValue(RequestContext value) {
* the end of {@link ContextSnapshot.Scope}. If reactor Context does not
* contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
* this method to remove the value from {@link ThreadLocal}.
- *
* Please note that {@link RequestContextUtil#pop()} return {@link AutoCloseable} instance,
* but it is not used in `Try with Resources` syntax. this is because {@link ContextSnapshot.Scope}
* will handle the {@link AutoCloseable} instance returned by {@link RequestContextUtil#pop()}.
@@ -131,7 +128,6 @@ public void restore(RequestContext previousValue) {
* the end of {@link ContextSnapshot.Scope}. If reactor Context does not
* contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
* this method to remove the value from {@link ThreadLocal}.
- *
* Please note that {@link RequestContextUtil#pop()} return {@link AutoCloseable} instance,
* but it is not used in `Try with Resources` syntax. this is because {@link ContextSnapshot.Scope}
* will handle the {@link AutoCloseable} instance returned by {@link RequestContextUtil#pop()}.
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
index 0e79cd96331..24e2c33027e 100644
--- a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
+++ b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
@@ -36,7 +36,6 @@ public final class RequestContextPropagationHook {
* Enable
* Context-propagation to keep {@link RequestContext} during
* Reactor operations.
- *
* Please note that enable {@link RequestContextPropagationHook} at the
* start of the application. otherwise, {@link RequestContext} may not be keep.
*/
From 64cb1968e91450669b22ff7971a0824016ad5baa Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 20:15:29 +0900
Subject: [PATCH 11/48] fix test fail
---
.../armeria/common/RequestContextAccessorTest.java | 14 ++++++++++++++
1 file changed, 14 insertions(+)
diff --git a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
index 57453bd276e..ac3b41f2c1d 100644
--- a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
@@ -35,15 +35,29 @@
import java.util.List;
import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import com.linecorp.armeria.client.ClientRequestContext;
+import com.linecorp.armeria.internal.common.RequestContextUtil;
import io.micrometer.context.ContextRegistry;
import io.micrometer.context.ThreadLocalAccessor;
class RequestContextAccessorTest {
+ /* Should clean up on RequestContext.
+ * because some test case does not clean up on RequestContext,
+ * and it will affect other tests if test are executed parallely.
+ */
+
+ @AfterEach
+ @SuppressWarnings("MustBeClosedChecker")
+ void cleanUp() {
+ RequestContextUtil.pop();
+ }
+
@Test
void should_be_loaded_by_SPI() {
final ContextRegistry ctxRegistry = ContextRegistry.getInstance();
From 9b32927f45d7cc9fb141c8849dfab628453a69d5 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 20:21:35 +0900
Subject: [PATCH 12/48] remove unused imported
---
.../com/linecorp/armeria/common/RequestContextAccessorTest.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
index ac3b41f2c1d..dacd012180f 100644
--- a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
@@ -36,7 +36,6 @@
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import com.linecorp.armeria.client.ClientRequestContext;
From 6d4762a3c25e459111a157acda91236276df472c Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Fri, 12 Apr 2024 23:16:14 +0900
Subject: [PATCH 13/48] Add test cases
---
.../RequestContextPropagationFluxTest.java | 25 +++++++++++++++++++
.../RequestContextPropagationMonoTest.java | 25 +++++++++++++++++++
2 files changed, 50 insertions(+)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index c6d3fdae7f6..7a99fb55543 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -397,6 +397,31 @@ void subscriberContextIsNotMissing() {
.verifyComplete();
}
+ @Test
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperation() {
+ // Given
+ final ClientRequestContext ctx = newContext();
+ final Flux flux;
+
+ // When
+ flux = Flux.just("Hello", "Hi")
+ .delayElements(Duration.ofMillis(1000))
+ .map(s -> {
+ if (s.equals("Hello")) {
+ throw new RuntimeException();
+ }
+ return s;
+ })
+ .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+
+ // Then
+ StepVerifier.create(flux)
+ .expectError(RuntimeException.class)
+ .verify();
+
+ assertThat(ctxExists(ctx)).isFalse();
+ }
+
private static Flux addCallbacks(Flux flux, ClientRequestContext ctx) {
return flux.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index e72377e1a54..611037df9b5 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -227,6 +227,31 @@ void subscriberContextIsNotMissing() {
.verifyComplete();
}
+ @Test
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperation() {
+ // Given
+ final ClientRequestContext ctx = newContext();
+ final Mono mono;
+
+ // When
+ mono = Mono.just("Hello")
+ .delayElement(Duration.ofMillis(1000))
+ .map(s -> {
+ if (s.equals("Hello")) {
+ throw new RuntimeException();
+ }
+ return s;
+ })
+ .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+
+ // Then
+ StepVerifier.create(mono)
+ .expectError(RuntimeException.class)
+ .verify();
+
+ assertThat(ctxExists(ctx)).isFalse();
+ }
+
static Subscription noopSubscription() {
return new Subscription() {
@Override
From 42097563ce5f748f018232d5e2ec180b08bdf329 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Sat, 13 Apr 2024 17:19:45 +0900
Subject: [PATCH 14/48] Add test case
---
.../RequestContextPropagationFluxTest.java | 26 ++++++++++++++++++-
.../RequestContextPropagationMonoTest.java | 26 ++++++++++++++++++-
2 files changed, 50 insertions(+), 2 deletions(-)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 7a99fb55543..83f224be2c2 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -398,7 +398,7 @@ void subscriberContextIsNotMissing() {
}
@Test
- void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperation() {
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread() {
// Given
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -422,6 +422,30 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperation() {
assertThat(ctxExists(ctx)).isFalse();
}
+ @Test
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnMainThread() {
+ // Given
+ final ClientRequestContext ctx = newContext();
+ final Flux flux;
+
+ // When
+ flux = Flux.just("Hello", "Hi")
+ .map(s -> {
+ if (s.equals("Hello")) {
+ throw new RuntimeException();
+ }
+ return s;
+ })
+ .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+
+ // Then
+ StepVerifier.create(flux)
+ .expectError(RuntimeException.class)
+ .verify();
+
+ assertThat(ctxExists(ctx)).isFalse();
+ }
+
private static Flux addCallbacks(Flux flux, ClientRequestContext ctx) {
return flux.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 611037df9b5..35558e485f2 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -228,7 +228,7 @@ void subscriberContextIsNotMissing() {
}
@Test
- void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperation() {
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread() {
// Given
final ClientRequestContext ctx = newContext();
final Mono mono;
@@ -252,6 +252,30 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperation() {
assertThat(ctxExists(ctx)).isFalse();
}
+ @Test
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnMainThread() {
+ // Given
+ final ClientRequestContext ctx = newContext();
+ final Mono mono;
+
+ // When
+ mono = Mono.just("Hello")
+ .map(s -> {
+ if (s.equals("Hello")) {
+ throw new RuntimeException();
+ }
+ return s;
+ })
+ .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+
+ // Then
+ StepVerifier.create(mono)
+ .expectError(RuntimeException.class)
+ .verify();
+
+ assertThat(ctxExists(ctx)).isFalse();
+ }
+
static Subscription noopSubscription() {
return new Subscription() {
@Override
From 747fa364bb286c735970fdef21beec21893e8d06 Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Wed, 17 Apr 2024 18:57:36 +0900
Subject: [PATCH 15/48] Update
core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
Co-authored-by: minux
---
.../com/linecorp/armeria/common/RequestContextAccessor.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index 2a040498b56..9a180328dec 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -75,7 +75,7 @@ public static String accessorKey() {
*/
@Override
public RequestContext getValue() {
- return RequestContextUtil.get();
+ return RequestContext.currentOrNull();
}
/**
From 83a84b0b2699995c85e3a6912de637e783c56a19 Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Wed, 17 Apr 2024 18:57:43 +0900
Subject: [PATCH 16/48] Update
core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
Co-authored-by: minux
---
.../com/linecorp/armeria/common/RequestContextAccessor.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index 9a180328dec..8e48d927979 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -90,7 +90,7 @@ public RequestContext getValue() {
*/
@Override
public void setValue(RequestContext value) {
- RequestContextUtil.getAndSet(value);
+ value.push();
}
/**
From 3bdbd3ebc086508ed531a4a5d3f7455cf8df025d Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Wed, 17 Apr 2024 18:57:49 +0900
Subject: [PATCH 17/48] Update
core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
Co-authored-by: minux
---
.../com/linecorp/armeria/common/RequestContextAccessor.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index 8e48d927979..408b248be3d 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -120,7 +120,7 @@ public void setValue() {
*/
@Override
public void restore(RequestContext previousValue) {
- RequestContextUtil.getAndSet(previousValue);
+ previousValue.push();
}
/**
From 767bd9ac60e33fc8c2eed04718de4e296db87239 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 17 Apr 2024 19:29:54 +0900
Subject: [PATCH 18/48] Fix lint error and test case
---
.../linecorp/armeria/common/RequestContextAccessor.java | 2 ++
.../reactor3/RequestContextPropagationFluxTest.java | 9 ++++++++-
2 files changed, 10 insertions(+), 1 deletion(-)
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index 408b248be3d..1ac0d59892a 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -89,6 +89,7 @@ public RequestContext getValue() {
* to revert the {@link RequestContextStorage} to its original state.
*/
@Override
+ @SuppressWarnings("MustBeClosedChecker")
public void setValue(RequestContext value) {
value.push();
}
@@ -119,6 +120,7 @@ public void setValue() {
* to revert the {@link RequestContextStorage} to its original state.
*/
@Override
+ @SuppressWarnings("MustBeClosedChecker")
public void restore(RequestContext previousValue) {
previousValue.push();
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 83f224be2c2..69436041e17 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -405,6 +405,7 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
// When
flux = Flux.just("Hello", "Hi")
+ .subscribeOn(Schedulers.single())
.delayElements(Duration.ofMillis(1000))
.map(s -> {
if (s.equals("Hello")) {
@@ -419,7 +420,13 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
.expectError(RuntimeException.class)
.verify();
- assertThat(ctxExists(ctx)).isFalse();
+ Flux toVerifyFlux = Flux.just("Dummy")
+ .subscribeOn(Schedulers.single())
+ .doOnNext(s -> assertThat(ctxExists(ctx)).isFalse());
+
+ StepVerifier.create(toVerifyFlux)
+ .expectNext("Dummy")
+ .verifyComplete();
}
@Test
From 88b14754380af5ce7bc76551736d1a72a1deaf55 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 17 Apr 2024 20:29:07 +0900
Subject: [PATCH 19/48] resolve lint error
---
.../common/reactor3/RequestContextPropagationFluxTest.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 69436041e17..95950c89d90 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -420,7 +420,7 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
.expectError(RuntimeException.class)
.verify();
- Flux toVerifyFlux = Flux.just("Dummy")
+ final Flux toVerifyFlux = Flux.just("Dummy")
.subscribeOn(Schedulers.single())
.doOnNext(s -> assertThat(ctxExists(ctx)).isFalse());
From 4a3ebfc375e933efc8f68d5cb325304909aa4e32 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Sun, 21 Apr 2024 19:59:02 +0900
Subject: [PATCH 20/48] remove RequestContextPropagationHooks.
---
.../RequestContextPropagationHook.java | 71 -------------------
.../RequestContextPropagationFluxTest.java | 5 +-
.../RequestContextPropagationMonoTest.java | 5 +-
3 files changed, 6 insertions(+), 75 deletions(-)
delete mode 100644 reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
diff --git a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java b/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
deleted file mode 100644
index 24e2c33027e..00000000000
--- a/reactor3/src/main/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationHook.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2020 LINE Corporation
- *
- * LINE Corporation licenses this file to you under the Apache License,
- * version 2.0 (the "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at:
- *
- * https://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 com.linecorp.armeria.common.reactor3;
-
-import com.linecorp.armeria.common.RequestContext;
-
-import reactor.core.publisher.Flux;
-import reactor.core.publisher.Hooks;
-import reactor.core.publisher.Mono;
-
-/**
- * Utility class to keep {@link RequestContext} during
- * Reactor operations
- * with
- * Context-propagation.
- */
-public final class RequestContextPropagationHook {
-
- private static volatile boolean enabled;
-
- /**
- * Enable
- * Context-propagation to keep {@link RequestContext} during
- * Reactor operations.
- * Please note that enable {@link RequestContextPropagationHook} at the
- * start of the application. otherwise, {@link RequestContext} may not be keep.
- */
- public static synchronized void enable() {
- if (enabled) {
- return;
- }
- Hooks.enableAutomaticContextPropagation();
- enabled = true;
- }
-
- /**
- * It returns whether the {@link RequestContextPropagationHook} is enabled.
- */
- public static boolean isEnabled() {
- return enabled;
- }
-
- /**
- * It disable {@link RequestContextPropagationHook}. {@link RequestContext}
- * will not be keep during both {@link Mono} and {@link Flux} Operations.
- */
- public static synchronized void disable() {
- if (!enabled) {
- return;
- }
-
- Hooks.disableAutomaticContextPropagation();
- enabled = false;
- }
-
- private RequestContextPropagationHook() {}
-}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 95950c89d90..e2d41731155 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -37,6 +37,7 @@
import reactor.core.Disposable;
import reactor.core.publisher.ConnectableFlux;
import reactor.core.publisher.Flux;
+import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
@@ -48,12 +49,12 @@ class RequestContextPropagationFluxTest {
@BeforeAll
static void setUp() {
- RequestContextPropagationHook.enable();
+ Hooks.enableAutomaticContextPropagation();
}
@AfterAll
static void tearDown() {
- RequestContextPropagationHook.disable();
+ Hooks.disableAutomaticContextPropagation();
}
@Test
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 35558e485f2..cc15972395f 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -33,6 +33,7 @@
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
+import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
@@ -45,12 +46,12 @@ class RequestContextPropagationMonoTest {
@BeforeAll
static void setUp() {
- RequestContextPropagationHook.enable();
+ Hooks.enableAutomaticContextPropagation();
}
@AfterAll
static void tearDown() {
- RequestContextPropagationHook.disable();
+ Hooks.disableAutomaticContextPropagation();
}
@Test
From 6a3d4f45a20d46983d0811c74a165ccb85966027 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Sun, 21 Apr 2024 20:08:58 +0900
Subject: [PATCH 21/48] remove init context for Stepverifier.
---
.../RequestContextPropagationFluxTest.java | 100 +++++++-----------
.../RequestContextPropagationMonoTest.java | 61 ++++-------
2 files changed, 63 insertions(+), 98 deletions(-)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index e2d41731155..21b3c10155a 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -69,9 +69,8 @@ void fluxError() {
return new AnticipatedException();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
+ StepVerifier.create(flux)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
}
@Test
@@ -86,9 +85,8 @@ void fluxFromPublisher() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -103,9 +101,8 @@ void fluxCreate() {
s.complete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -119,9 +116,8 @@ void fluxCreate_error() {
s.error(new AnticipatedException());
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
+ StepVerifier.create(flux)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
}
@Test
@@ -137,10 +133,9 @@ void fluxConcat() {
return "bar";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
- .expectNextMatches(s -> ctxExists(ctx) && "bar".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
.verifyComplete();
}
@@ -154,9 +149,8 @@ void fluxDefer() {
return Flux.just("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -170,9 +164,8 @@ void fluxFromStream() {
return Stream.of("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -186,9 +179,8 @@ void fluxCombineLatest() {
return a;
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -203,9 +195,8 @@ void fluxGenerate() {
s.complete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -226,10 +217,9 @@ void fluxMerge() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
- .expectNextMatches(s -> ctxExists(ctx) && "bar".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
.verifyComplete();
}
@@ -244,9 +234,8 @@ void fluxPush() {
s.complete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -267,9 +256,8 @@ void fluxSwitchOnNext() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -283,9 +271,8 @@ void fluxZip() {
return foo;
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -299,10 +286,9 @@ void fluxInterval() {
return Mono.just("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -322,11 +308,10 @@ void fluxConcatDelayError() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
- .expectNextMatches(s -> ctxExists(ctx) && "bar".equals(s))
- .verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
}
@Test
@@ -341,9 +326,8 @@ void fluxTransform() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -356,9 +340,8 @@ void connectableFlux() {
flux = addCallbacks(connectableFlux.autoConnect(2).publishOn(Schedulers.single()), ctx);
flux.subscribe().dispose();
- StepVerifier.create(flux, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -392,9 +375,8 @@ void subscriberContextIsNotMissing() {
});
final Flux flux1 = flux.contextWrite(reactorCtx -> reactorCtx.put("foo", "bar"));
- StepVerifier.create(flux1, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "baz".equals(s))
+ StepVerifier.create(flux1)
+ .expectNextMatches(s -> "baz".equals(s))
.verifyComplete();
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index cc15972395f..267e235ed05 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -37,7 +37,6 @@
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
-import reactor.test.StepVerifierOptions;
import reactor.util.context.Context;
import reactor.util.function.Tuple2;
@@ -63,9 +62,8 @@ void monoCreate_success() {
sink.success("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -78,9 +76,8 @@ void monoCreate_error() {
sink.error(new AnticipatedException());
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
+ StepVerifier.create(mono)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
}
@Test
@@ -92,9 +89,8 @@ void monoCreate_currentContext() {
sink.success("foo");
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -107,9 +103,8 @@ void monoDefer() {
return "foo";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -124,9 +119,8 @@ void monoFromPublisher() {
s.onComplete();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -139,9 +133,8 @@ void monoError() {
return new AnticipatedException();
}).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .verifyErrorMatches(t -> ctxExists(ctx) && t instanceof AnticipatedException);
+ StepVerifier.create(mono)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
}
@Test
@@ -154,9 +147,8 @@ void monoFirst() {
return "foo";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -168,9 +160,8 @@ void monoFromFuture() {
final Mono mono;
mono = addCallbacks(Mono.fromFuture(future).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -185,9 +176,8 @@ void monoDelay() {
return "foo";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "foo".equals(s))
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
@@ -205,10 +195,8 @@ void monoZip() {
return "bar";
})).publishOn(Schedulers.single()), ctx);
- StepVerifier.create(mono, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(t -> ctxExists(ctx) &&
- "foo".equals(t.getT1()) && "bar".equals(t.getT2()))
+ StepVerifier.create(mono)
+ .expectNextMatches(t -> "foo".equals(t.getT1()) && "bar".equals(t.getT2()))
.verifyComplete();
}
@@ -222,9 +210,8 @@ void subscriberContextIsNotMissing() {
});
final Mono mono1 = mono.contextWrite(reactorCtx -> reactorCtx.put("foo", "bar"));
- StepVerifier.create(mono1, initialReactorContext(ctx))
- .expectSubscriptionMatches(s -> ctxExists(ctx))
- .expectNextMatches(s -> ctxExists(ctx) && "baz".equals(s))
+ StepVerifier.create(mono1)
+ .expectNextMatches(s -> "baz".equals(s))
.verifyComplete();
}
@@ -309,8 +296,4 @@ private static Mono addCallbacks(Mono mono, ClientRequestContext ctx)
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
}
- private static StepVerifierOptions initialReactorContext(ClientRequestContext ctx) {
- final Context reactorCtx = Context.of(RequestContextAccessor.accessorKey(), ctx);
- return StepVerifierOptions.create().withInitialContext(reactorCtx);
- }
}
From 7d2970d315415950bcc93d647d1d12acfcc479bd Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Sun, 21 Apr 2024 21:08:27 +0900
Subject: [PATCH 22/48] Add test cases for contextCapture().
---
.../RequestContextPropagationFluxTest.java | 506 ++++++++++++------
.../RequestContextPropagationMonoTest.java | 281 +++++++---
2 files changed, 559 insertions(+), 228 deletions(-)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 21b3c10155a..25beaff986c 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -27,10 +27,14 @@
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
import org.reactivestreams.Publisher;
import com.linecorp.armeria.client.ClientRequestContext;
import com.linecorp.armeria.common.RequestContextAccessor;
+import com.linecorp.armeria.common.util.SafeCloseable;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
@@ -41,12 +45,19 @@
import reactor.core.publisher.Mono;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
-import reactor.test.StepVerifierOptions;
import reactor.util.context.Context;
@GenerateNativeImageTrace
class RequestContextPropagationFluxTest {
+ static Stream provideContextWriteAndCaptureTestCase() {
+ return Stream.of(
+ // shouldContextWrite, shouldContextCapture.
+ Arguments.of(true, false),
+ Arguments.of(false, true)
+ );
+ }
+
@BeforeAll
static void setUp() {
Hooks.enableAutomaticContextPropagation();
@@ -57,8 +68,10 @@ static void tearDown() {
Hooks.disableAutomaticContextPropagation();
}
- @Test
- void fluxError() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxError(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -67,14 +80,23 @@ void fluxError() {
// After publishOn(), ctxExists(ctx) should be false.
// On the other hand, it should be True due to ContextPropagation.
return new AnticipatedException();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
+ } else {
+ StepVerifier.create(flux)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
}
- @Test
- void fluxFromPublisher() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxFromPublisher(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -83,15 +105,25 @@ void fluxFromPublisher() {
s.onSubscribe(noopSubscription());
s.onNext("foo");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxCreate() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxCreate(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -99,29 +131,48 @@ void fluxCreate() {
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxCreate_error() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxCreate_error(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.create(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.error(new AnticipatedException());
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
+ } else {
+ StepVerifier.create(flux)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
}
- @Test
- void fluxConcat() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxConcat(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -131,61 +182,102 @@ void fluxConcat() {
}), Mono.fromCallable(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "bar";
- })).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
- .verifyComplete();
+ })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxDefer() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxDefer(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.defer(() -> {
assertThat(ctxExists(ctx)).isTrue();
return Flux.just("foo");
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxFromStream() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxFromStream(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.fromStream(() -> {
assertThat(ctxExists(ctx)).isTrue();
return Stream.of("foo");
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxCombineLatest() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxCombineLatest(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.combineLatest(Mono.just("foo"), Mono.just("bar"), (a, b) -> {
assertThat(ctxExists(ctx)).isTrue();
return a;
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxGenerate() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxGenerate(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -193,15 +285,25 @@ void fluxGenerate() {
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxMerge() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxMerge(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -215,16 +317,27 @@ void fluxMerge() {
s.onSubscribe(noopSubscription());
s.onNext("bar");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxPush() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxPush(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -232,15 +345,25 @@ void fluxPush() {
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxSwitchOnNext() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxSwitchOnNext(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -254,46 +377,77 @@ void fluxSwitchOnNext() {
s1.onComplete();
});
s.onComplete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxZip() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxZip(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.zip(Mono.just("foo"), Mono.just("bar"), (foo, bar) -> {
assertThat(ctxExists(ctx)).isTrue();
return foo;
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxInterval() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxInterval(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.interval(Duration.ofMillis(100)).take(2).concatMap(a -> {
assertThat(ctxExists(ctx)).isTrue();
return Mono.just("foo");
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void fluxConcatDelayError() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxConcatDelayError(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -306,16 +460,27 @@ void fluxConcatDelayError() {
s.onSubscribe(noopSubscription());
s.onNext("bar");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
- .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches(s -> "bar".equals(s))
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
}
- @Test
- void fluxTransform() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void fluxTransform(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -324,44 +489,80 @@ void fluxTransform() {
s.onSubscribe(noopSubscription());
s.onNext(fooFlux.blockFirst());
s.onComplete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void connectableFlux() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void connectableFlux(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
final ConnectableFlux connectableFlux = Flux.just("foo").publish();
- flux = addCallbacks(connectableFlux.autoConnect(2).publishOn(Schedulers.single()), ctx);
-
- flux.subscribe().dispose();
- StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ flux = addCallbacks(connectableFlux.autoConnect(2).publishOn(Schedulers.single()),
+ ctx,
+ shouldContextWrite,
+ shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ flux.subscribe().dispose();
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ flux.subscribe().dispose();
+ StepVerifier.create(flux)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void connectableFlux_dispose() throws InterruptedException {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void connectableFlux_dispose(boolean shouldContextWrite,
+ boolean shouldContextCapture) throws InterruptedException {
final ClientRequestContext ctx = newContext();
final Flux flux;
final ConnectableFlux connectableFlux = Flux.just("foo").publish();
flux = addCallbacks(connectableFlux.autoConnect(2, disposable -> {
assertThat(ctxExists(ctx)).isTrue();
- }).publishOn(Schedulers.newSingle("aaa")), ctx);
-
- final Disposable disposable1 = flux.subscribe();
- await().pollDelay(Duration.ofMillis(200)).until(() -> !disposable1.isDisposed());
- final Disposable disposable2 = flux.subscribe();
- await().untilAsserted(() -> {
- assertThat(disposable1.isDisposed()).isTrue();
- assertThat(disposable2.isDisposed()).isTrue();
- });
+ }).publishOn(Schedulers.newSingle("aaa")), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ final Disposable disposable1 = flux.subscribe();
+ await().pollDelay(Duration.ofMillis(200)).until(() -> !disposable1.isDisposed());
+ final Disposable disposable2 = flux.subscribe();
+ await().untilAsserted(() -> {
+ assertThat(disposable1.isDisposed()).isTrue();
+ assertThat(disposable2.isDisposed()).isTrue();
+ });
+ }
+ } else {
+ final Disposable disposable1 = flux.subscribe();
+ await().pollDelay(Duration.ofMillis(200)).until(() -> !disposable1.isDisposed());
+ final Disposable disposable2 = flux.subscribe();
+ await().untilAsserted(() -> {
+ assertThat(disposable1.isDisposed()).isTrue();
+ assertThat(disposable2.isDisposed()).isTrue();
+ });
+ }
}
@Test
@@ -436,21 +637,28 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnMainThread() {
assertThat(ctxExists(ctx)).isFalse();
}
- private static Flux addCallbacks(Flux flux, ClientRequestContext ctx) {
- return flux.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
- .doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
- .doOnRequest(l -> assertThat(ctxExists(ctx)).isTrue())
- .doOnNext(foo -> assertThat(ctxExists(ctx)).isTrue())
- .doOnComplete(() -> assertThat(ctxExists(ctx)).isTrue())
- .doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
- .doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
- .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue())
- .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ private static Flux addCallbacks(Flux flux0,
+ ClientRequestContext ctx,
+ boolean shouldContextWrite,
+ boolean shouldContextCapture) {
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
- }
-
- private static StepVerifierOptions initialReactorContext(ClientRequestContext ctx) {
- final Context reactorCtx = Context.of(RequestContextAccessor.accessorKey(), ctx);
- return StepVerifierOptions.create().withInitialContext(reactorCtx);
+ final Flux flux = flux0.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnRequest(l -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnNext(foo -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnComplete(() -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
+ .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
+
+ if (shouldContextWrite) {
+ return flux.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ }
+
+ if (shouldContextCapture) {
+ return flux.contextCapture();
+ }
+
+ return flux;
}
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 267e235ed05..430eb2d2166 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -19,10 +19,14 @@
import java.time.Duration;
import java.util.concurrent.CompletableFuture;
+import java.util.stream.Stream;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
import org.reactivestreams.Subscription;
import com.linecorp.armeria.client.ClientRequestContext;
@@ -30,6 +34,7 @@
import com.linecorp.armeria.common.HttpRequest;
import com.linecorp.armeria.common.RequestContext;
import com.linecorp.armeria.common.RequestContextAccessor;
+import com.linecorp.armeria.common.util.SafeCloseable;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
@@ -43,6 +48,14 @@
@GenerateNativeImageTrace
class RequestContextPropagationMonoTest {
+ static Stream provideContextWriteAndCaptureTestCase() {
+ return Stream.of(
+ // shouldContextWrite, shouldContextCapture.
+ Arguments.of(true, false),
+ Arguments.of(false, true)
+ );
+ }
+
@BeforeAll
static void setUp() {
Hooks.enableAutomaticContextPropagation();
@@ -53,63 +66,104 @@ static void tearDown() {
Hooks.disableAutomaticContextPropagation();
}
- @Test
- void monoCreate_success() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoCreate_success(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.success("foo");
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void monoCreate_error() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoCreate_error(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.error(new AnticipatedException());
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
+ } else {
+ StepVerifier.create(mono)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
}
- @Test
- void monoCreate_currentContext() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoCreate_currentContext(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.success("foo");
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void monoDefer() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoDefer(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.defer(() -> Mono.fromSupplier(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
- })).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void monoFromPublisher() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoFromPublisher(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.from(s -> {
@@ -117,56 +171,97 @@ void monoFromPublisher() {
s.onSubscribe(noopSubscription());
s.onNext("foo");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void monoError() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoError(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.error(() -> {
assertThat(ctxExists(ctx)).isTrue();
return new AnticipatedException();
- }).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
+ } else {
+ StepVerifier.create(mono)
+ .verifyErrorMatches(t -> t instanceof AnticipatedException);
+ }
}
- @Test
- void monoFirst() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoFirst(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.firstWithSignal(Mono.delay(Duration.ofMillis(1000)).then(Mono.just("bar")),
Mono.fromCallable(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
- })).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ }))
+ .publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void monoFromFuture() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoFromFuture(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final CompletableFuture future = new CompletableFuture<>();
future.complete("foo");
final ClientRequestContext ctx = newContext();
final Mono mono;
- mono = addCallbacks(Mono.fromFuture(future).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ mono = addCallbacks(Mono.fromFuture(future)
+ .publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void monoDelay() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoDelay(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final CompletableFuture future = new CompletableFuture<>();
future.complete("foo");
final ClientRequestContext ctx = newContext();
@@ -174,15 +269,25 @@ void monoDelay() {
mono = addCallbacks(Mono.delay(Duration.ofMillis(100)).then(Mono.fromCallable(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
- })).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
- .verifyComplete();
+ })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(s -> "foo".equals(s))
+ .verifyComplete();
+ }
}
- @Test
- void monoZip() {
+ @ParameterizedTest
+ @MethodSource("provideContextWriteAndCaptureTestCase")
+ void monoZip(boolean shouldContextWrite,
+ boolean shouldContextCapture) {
final CompletableFuture future = new CompletableFuture<>();
future.complete("foo");
final ClientRequestContext ctx = newContext();
@@ -193,11 +298,19 @@ void monoZip() {
}), Mono.fromSupplier(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "bar";
- })).publishOn(Schedulers.single()), ctx);
-
- StepVerifier.create(mono)
- .expectNextMatches(t -> "foo".equals(t.getT1()) && "bar".equals(t.getT2()))
- .verifyComplete();
+ })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+
+ if (shouldContextCapture) {
+ try (SafeCloseable ignored = ctx.push()) {
+ StepVerifier.create(mono)
+ .expectNextMatches(t -> "foo".equals(t.getT1()) && "bar".equals(t.getT2()))
+ .verifyComplete();
+ }
+ } else {
+ StepVerifier.create(mono)
+ .expectNextMatches(t -> "foo".equals(t.getT1()) && "bar".equals(t.getT2()))
+ .verifyComplete();
+ }
}
@Test
@@ -283,17 +396,27 @@ static ClientRequestContext newContext() {
.build();
}
- private static Mono addCallbacks(Mono mono, ClientRequestContext ctx) {
- return mono.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
- .doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
- .doOnRequest(l -> assertThat(ctxExists(ctx)).isTrue())
- .doOnNext(foo -> assertThat(ctxExists(ctx)).isTrue())
- .doOnSuccess(t -> assertThat(ctxExists(ctx)).isTrue())
- .doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
- .doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
- .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue())
- .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ private static Mono addCallbacks(Mono mono0, ClientRequestContext ctx,
+ boolean shouldContextWrite,
+ boolean shouldContextCapture) {
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
+ final Mono mono = mono0.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnRequest(l -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnNext(foo -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnSuccess(t -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
+ .doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
+
+ if (shouldContextWrite) {
+ return mono.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ }
+
+ if (shouldContextCapture) {
+ return mono.contextCapture();
+ }
+
+ return mono;
}
-
}
From 24c6f387a02febdc91b69c22f20254ef56d59300 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Sun, 21 Apr 2024 23:18:05 +0900
Subject: [PATCH 23/48] Should not ctx in main thread.
---
.../RequestContextPropagationFluxTest.java | 20 +++++++++++++++++++
.../RequestContextPropagationMonoTest.java | 12 +++++++++++
2 files changed, 32 insertions(+)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 25beaff986c..0930ba798fb 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -91,6 +91,7 @@ void fluxError(boolean shouldContextWrite,
StepVerifier.create(flux)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -118,6 +119,7 @@ void fluxFromPublisher(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -144,6 +146,7 @@ void fluxCreate(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -167,6 +170,7 @@ void fluxCreate_error(boolean shouldContextWrite,
StepVerifier.create(flux)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -197,6 +201,7 @@ void fluxConcat(boolean shouldContextWrite,
.expectNextMatches(s -> "bar".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -222,6 +227,7 @@ void fluxDefer(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -247,6 +253,7 @@ void fluxFromStream(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -272,6 +279,7 @@ void fluxCombineLatest(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -298,6 +306,7 @@ void fluxGenerate(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -332,6 +341,7 @@ void fluxMerge(boolean shouldContextWrite,
.expectNextMatches(s -> "bar".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -358,6 +368,7 @@ void fluxPush(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -390,6 +401,7 @@ void fluxSwitchOnNext(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -415,6 +427,7 @@ void fluxZip(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -442,6 +455,7 @@ void fluxInterval(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -502,6 +516,7 @@ void fluxTransform(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -530,6 +545,7 @@ void connectableFlux(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -563,6 +579,7 @@ void connectableFlux_dispose(boolean shouldContextWrite,
assertThat(disposable2.isDisposed()).isTrue();
});
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@Test
@@ -579,6 +596,7 @@ void subscriberContextIsNotMissing() {
StepVerifier.create(flux1)
.expectNextMatches(s -> "baz".equals(s))
.verifyComplete();
+ assertThat(ctxExists(ctx)).isFalse();
}
@Test
@@ -611,6 +629,7 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
StepVerifier.create(toVerifyFlux)
.expectNext("Dummy")
.verifyComplete();
+ assertThat(ctxExists(ctx)).isFalse();
}
@Test
@@ -635,6 +654,7 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnMainThread() {
.verify();
assertThat(ctxExists(ctx)).isFalse();
+ assertThat(ctxExists(ctx)).isFalse();
}
private static Flux addCallbacks(Flux flux0,
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 430eb2d2166..09ee26c49e5 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -15,6 +15,7 @@
*/
package com.linecorp.armeria.common.reactor3;
+import static com.linecorp.armeria.common.reactor3.ContextAwareMonoTest.ctxExists;
import static org.assertj.core.api.Assertions.assertThat;
import java.time.Duration;
@@ -88,6 +89,7 @@ void monoCreate_success(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -110,6 +112,7 @@ void monoCreate_error(boolean shouldContextWrite,
StepVerifier.create(mono)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -134,6 +137,7 @@ void monoCreate_currentContext(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -158,6 +162,7 @@ void monoDefer(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -184,6 +189,7 @@ void monoFromPublisher(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -206,6 +212,7 @@ void monoError(boolean shouldContextWrite,
StepVerifier.create(mono)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -232,6 +239,7 @@ void monoFirst(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -256,6 +264,7 @@ void monoFromFuture(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -282,6 +291,7 @@ void monoDelay(boolean shouldContextWrite,
.expectNextMatches(s -> "foo".equals(s))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
@@ -311,6 +321,7 @@ void monoZip(boolean shouldContextWrite,
.expectNextMatches(t -> "foo".equals(t.getT1()) && "bar".equals(t.getT2()))
.verifyComplete();
}
+ assertThat(ctxExists(ctx)).isFalse();
}
@Test
@@ -326,6 +337,7 @@ void subscriberContextIsNotMissing() {
StepVerifier.create(mono1)
.expectNextMatches(s -> "baz".equals(s))
.verifyComplete();
+ assertThat(ctxExists(ctx)).isFalse();
}
@Test
From 4a0081448b48a172a4d2522309a8b16bb7cb6e58 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Mon, 22 Apr 2024 14:15:50 +0900
Subject: [PATCH 24/48] apply review
---
.../common/RequestContextAccessorTest.java | 18 +-
.../RequestContextPropagationFluxTest.java | 323 ++++++++----------
.../RequestContextPropagationMonoTest.java | 184 ++++------
3 files changed, 215 insertions(+), 310 deletions(-)
diff --git a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
index dacd012180f..57a830c1844 100644
--- a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
@@ -35,7 +35,6 @@
import java.util.List;
import org.assertj.core.api.Assertions;
-import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import com.linecorp.armeria.client.ClientRequestContext;
@@ -46,17 +45,6 @@
class RequestContextAccessorTest {
- /* Should clean up on RequestContext.
- * because some test case does not clean up on RequestContext,
- * and it will affect other tests if test are executed parallely.
- */
-
- @AfterEach
- @SuppressWarnings("MustBeClosedChecker")
- void cleanUp() {
- RequestContextUtil.pop();
- }
-
@Test
void should_be_loaded_by_SPI() {
final ContextRegistry ctxRegistry = ContextRegistry.getInstance();
@@ -80,6 +68,7 @@ void should_return_expected_key() {
}
@Test
+ @SuppressWarnings("MustBeClosedChecker")
void should_success_set() {
// Given
final ClientRequestContext ctx = newContext();
@@ -91,6 +80,8 @@ void should_success_set() {
// Then
final RequestContext currentCtx = RequestContext.current();
assertThat(currentCtx).isEqualTo(ctx);
+
+ RequestContextUtil.pop();
}
@Test
@@ -119,6 +110,7 @@ void should_be_null_when_setValue() {
}
@Test
+ @SuppressWarnings("MustBeClosedChecker")
void should_be_restore_original_state_when_restore() {
// Given
final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
@@ -133,6 +125,8 @@ void should_be_restore_original_state_when_restore() {
final RequestContext reqCtx = RequestContext.currentOrNull();
assertThat(reqCtx).isNotNull();
assertThat(reqCtx).isEqualTo(previousCtx);
+
+ RequestContextUtil.pop();
}
@Test
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 0930ba798fb..7297117e3d6 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -22,14 +22,15 @@
import static org.awaitility.Awaitility.await;
import java.time.Duration;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Stream;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.Arguments;
-import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.CsvSource;
import org.reactivestreams.Publisher;
import com.linecorp.armeria.client.ClientRequestContext;
@@ -43,6 +44,7 @@
import reactor.core.publisher.Flux;
import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
+import reactor.core.scheduler.Scheduler;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
import reactor.util.context.Context;
@@ -50,14 +52,6 @@
@GenerateNativeImageTrace
class RequestContextPropagationFluxTest {
- static Stream provideContextWriteAndCaptureTestCase() {
- return Stream.of(
- // shouldContextWrite, shouldContextCapture.
- Arguments.of(true, false),
- Arguments.of(false, true)
- );
- }
-
@BeforeAll
static void setUp() {
Hooks.enableAutomaticContextPropagation();
@@ -69,20 +63,24 @@ static void tearDown() {
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxError(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxError(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
+ final AtomicBoolean atomicBoolean = new AtomicBoolean();
flux = addCallbacks(Flux.error(() -> {
- // This is called twice. after publishOn() and verifyErrorMatches()
- // After publishOn(), ctxExists(ctx) should be false.
- // On the other hand, it should be True due to ContextPropagation.
+ if (!atomicBoolean.getAndSet(true)) {
+ // Flux.error().publishOn() calls this error supplier immediately to see if it can retrieve
+ // the value via Callable.call() without ctx.
+ assertThat(ctxExists(ctx)).isFalse();
+ } else {
+ assertThat(ctxExists(ctx)).isTrue();
+ }
return new AnticipatedException();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
@@ -95,9 +93,8 @@ void fluxError(boolean shouldContextWrite,
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxFromPublisher(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxFromPublisher(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -106,26 +103,25 @@ void fluxFromPublisher(boolean shouldContextWrite,
s.onSubscribe(noopSubscription());
s.onNext("foo");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxCreate(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxCreate(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -133,35 +129,34 @@ void fluxCreate(boolean shouldContextWrite,
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxCreate_error(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxCreate_error(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.create(s -> {
assertThat(ctxExists(ctx)).isTrue();
s.error(new AnticipatedException());
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
@@ -174,9 +169,8 @@ void fluxCreate_error(boolean shouldContextWrite,
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxConcat(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxConcat(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -186,106 +180,102 @@ void fluxConcat(boolean shouldContextWrite,
}), Mono.fromCallable(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "bar";
- })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ })).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("bar"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("bar"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxDefer(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxDefer(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.defer(() -> {
assertThat(ctxExists(ctx)).isTrue();
return Flux.just("foo");
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxFromStream(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxFromStream(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.fromStream(() -> {
assertThat(ctxExists(ctx)).isTrue();
return Stream.of("foo");
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxCombineLatest(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxCombineLatest(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.combineLatest(Mono.just("foo"), Mono.just("bar"), (a, b) -> {
assertThat(ctxExists(ctx)).isTrue();
return a;
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxGenerate(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxGenerate(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -293,26 +283,25 @@ void fluxGenerate(boolean shouldContextWrite,
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxMerge(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxMerge(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -326,28 +315,27 @@ void fluxMerge(boolean shouldContextWrite,
s.onSubscribe(noopSubscription());
s.onNext("bar");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("bar"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("bar"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxPush(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxPush(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -355,26 +343,25 @@ void fluxPush(boolean shouldContextWrite,
assertThat(ctxExists(ctx)).isTrue();
s.next("foo");
s.complete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxSwitchOnNext(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxSwitchOnNext(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -388,80 +375,77 @@ void fluxSwitchOnNext(boolean shouldContextWrite,
s1.onComplete();
});
s.onComplete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxZip(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxZip(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.zip(Mono.just("foo"), Mono.just("bar"), (foo, bar) -> {
assertThat(ctxExists(ctx)).isTrue();
return foo;
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxInterval(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxInterval(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
flux = addCallbacks(Flux.interval(Duration.ofMillis(100)).take(2).concatMap(a -> {
assertThat(ctxExists(ctx)).isTrue();
return Mono.just("foo");
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxConcatDelayError(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxConcatDelayError(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -474,27 +458,26 @@ void fluxConcatDelayError(boolean shouldContextWrite,
s.onSubscribe(noopSubscription());
s.onNext("bar");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("bar"::equals)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
- .expectNextMatches(s -> "bar".equals(s))
+ .expectNextMatches("foo"::equals)
+ .expectNextMatches("bar"::equals)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
}
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void fluxTransform(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void fluxTransform(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
@@ -503,64 +486,61 @@ void fluxTransform(boolean shouldContextWrite,
s.onSubscribe(noopSubscription());
s.onNext(fooFlux.blockFirst());
s.onComplete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void connectableFlux(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void connectableFlux(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Flux flux;
final ConnectableFlux connectableFlux = Flux.just("foo").publish();
flux = addCallbacks(connectableFlux.autoConnect(2).publishOn(Schedulers.single()),
ctx,
- shouldContextWrite,
- shouldContextCapture);
+ useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
flux.subscribe().dispose();
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
flux.subscribe().dispose();
StepVerifier.create(flux)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void connectableFlux_dispose(boolean shouldContextWrite,
- boolean shouldContextCapture) throws InterruptedException {
+ @CsvSource({ "true", "false" })
+ void connectableFlux_dispose(boolean useContextCapture) throws InterruptedException {
final ClientRequestContext ctx = newContext();
final Flux flux;
final ConnectableFlux connectableFlux = Flux.just("foo").publish();
flux = addCallbacks(connectableFlux.autoConnect(2, disposable -> {
assertThat(ctxExists(ctx)).isTrue();
- }).publishOn(Schedulers.newSingle("aaa")), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.newSingle("aaa")), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
final Disposable disposable1 = flux.subscribe();
await().pollDelay(Duration.ofMillis(200)).until(() -> !disposable1.isDisposed());
@@ -594,23 +574,25 @@ void subscriberContextIsNotMissing() {
final Flux flux1 = flux.contextWrite(reactorCtx -> reactorCtx.put("foo", "bar"));
StepVerifier.create(flux1)
- .expectNextMatches(s -> "baz".equals(s))
+ .expectNextMatches("baz"::equals)
.verifyComplete();
assertThat(ctxExists(ctx)).isFalse();
}
@Test
- void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread() {
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread()
+ throws InterruptedException {
// Given
final ClientRequestContext ctx = newContext();
final Flux flux;
+ final Scheduler single = Schedulers.single();
// When
flux = Flux.just("Hello", "Hi")
- .subscribeOn(Schedulers.single())
+ .subscribeOn(single)
.delayElements(Duration.ofMillis(1000))
.map(s -> {
- if (s.equals("Hello")) {
+ if ("Hello".equals(s)) {
throw new RuntimeException();
}
return s;
@@ -622,45 +604,19 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
.expectError(RuntimeException.class)
.verify();
- final Flux toVerifyFlux = Flux.just("Dummy")
- .subscribeOn(Schedulers.single())
- .doOnNext(s -> assertThat(ctxExists(ctx)).isFalse());
-
- StepVerifier.create(toVerifyFlux)
- .expectNext("Dummy")
- .verifyComplete();
- assertThat(ctxExists(ctx)).isFalse();
- }
-
- @Test
- void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnMainThread() {
- // Given
- final ClientRequestContext ctx = newContext();
- final Flux flux;
-
- // When
- flux = Flux.just("Hello", "Hi")
- .map(s -> {
- if (s.equals("Hello")) {
- throw new RuntimeException();
- }
- return s;
- })
- .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
-
- // Then
- StepVerifier.create(flux)
- .expectError(RuntimeException.class)
- .verify();
+ final CountDownLatch latch = new CountDownLatch(1);
+ single.schedule(() -> {
+ assertThat(ctxExists(ctx)).isFalse();
+ latch.countDown();
+ });
+ latch.await();
assertThat(ctxExists(ctx)).isFalse();
- assertThat(ctxExists(ctx)).isFalse();
}
private static Flux addCallbacks(Flux flux0,
ClientRequestContext ctx,
- boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ boolean useContextCapture) {
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
final Flux flux = flux0.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
@@ -671,14 +627,9 @@ private static Flux addCallbacks(Flux flux0,
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
.doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
- if (shouldContextWrite) {
- return flux.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
- }
-
- if (shouldContextCapture) {
+ if (useContextCapture) {
return flux.contextCapture();
}
-
- return flux;
+ return flux.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
}
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 09ee26c49e5..5cfc4f55a0e 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -15,19 +15,17 @@
*/
package com.linecorp.armeria.common.reactor3;
-import static com.linecorp.armeria.common.reactor3.ContextAwareMonoTest.ctxExists;
import static org.assertj.core.api.Assertions.assertThat;
import java.time.Duration;
import java.util.concurrent.CompletableFuture;
-import java.util.stream.Stream;
+import java.util.concurrent.CountDownLatch;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.Arguments;
-import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.CsvSource;
import org.reactivestreams.Subscription;
import com.linecorp.armeria.client.ClientRequestContext;
@@ -41,6 +39,7 @@
import reactor.core.publisher.Hooks;
import reactor.core.publisher.Mono;
+import reactor.core.scheduler.Scheduler;
import reactor.core.scheduler.Schedulers;
import reactor.test.StepVerifier;
import reactor.util.context.Context;
@@ -49,14 +48,6 @@
@GenerateNativeImageTrace
class RequestContextPropagationMonoTest {
- static Stream provideContextWriteAndCaptureTestCase() {
- return Stream.of(
- // shouldContextWrite, shouldContextCapture.
- Arguments.of(true, false),
- Arguments.of(false, true)
- );
- }
-
@BeforeAll
static void setUp() {
Hooks.enableAutomaticContextPropagation();
@@ -68,42 +59,40 @@ static void tearDown() {
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoCreate_success(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoCreate_success(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.success("foo");
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoCreate_error(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoCreate_error(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.error(new AnticipatedException());
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
@@ -116,59 +105,56 @@ void monoCreate_error(boolean shouldContextWrite,
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoCreate_currentContext(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoCreate_currentContext(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.create(sink -> {
assertThat(ctxExists(ctx)).isTrue();
sink.success("foo");
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoDefer(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoDefer(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.defer(() -> Mono.fromSupplier(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
- })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ })).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoFromPublisher(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoFromPublisher(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.from(s -> {
@@ -176,34 +162,33 @@ void monoFromPublisher(boolean shouldContextWrite,
s.onSubscribe(noopSubscription());
s.onNext("foo");
s.onComplete();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoError(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoError(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.error(() -> {
assertThat(ctxExists(ctx)).isTrue();
return new AnticipatedException();
- }).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ }).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
.verifyErrorMatches(t -> t instanceof AnticipatedException);
@@ -216,9 +201,8 @@ void monoError(boolean shouldContextWrite,
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoFirst(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoFirst(boolean useContextCapture) {
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.firstWithSignal(Mono.delay(Duration.ofMillis(1000)).then(Mono.just("bar")),
@@ -226,51 +210,49 @@ void monoFirst(boolean shouldContextWrite,
assertThat(ctxExists(ctx)).isTrue();
return "foo";
}))
- .publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ .publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoFromFuture(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoFromFuture(boolean useContextCapture) {
final CompletableFuture future = new CompletableFuture<>();
future.complete("foo");
final ClientRequestContext ctx = newContext();
final Mono mono;
mono = addCallbacks(Mono.fromFuture(future)
- .publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ .publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoDelay(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoDelay(boolean useContextCapture) {
final CompletableFuture future = new CompletableFuture<>();
future.complete("foo");
final ClientRequestContext ctx = newContext();
@@ -278,26 +260,25 @@ void monoDelay(boolean shouldContextWrite,
mono = addCallbacks(Mono.delay(Duration.ofMillis(100)).then(Mono.fromCallable(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "foo";
- })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ })).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
} else {
StepVerifier.create(mono)
- .expectNextMatches(s -> "foo".equals(s))
+ .expectNextMatches("foo"::equals)
.verifyComplete();
}
assertThat(ctxExists(ctx)).isFalse();
}
@ParameterizedTest
- @MethodSource("provideContextWriteAndCaptureTestCase")
- void monoZip(boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ @CsvSource({ "true", "false" })
+ void monoZip(boolean useContextCapture) {
final CompletableFuture future = new CompletableFuture<>();
future.complete("foo");
final ClientRequestContext ctx = newContext();
@@ -308,9 +289,9 @@ void monoZip(boolean shouldContextWrite,
}), Mono.fromSupplier(() -> {
assertThat(ctxExists(ctx)).isTrue();
return "bar";
- })).publishOn(Schedulers.single()), ctx, shouldContextWrite, shouldContextCapture);
+ })).publishOn(Schedulers.single()), ctx, useContextCapture);
- if (shouldContextCapture) {
+ if (useContextCapture) {
try (SafeCloseable ignored = ctx.push()) {
StepVerifier.create(mono)
.expectNextMatches(t -> "foo".equals(t.getT1()) && "bar".equals(t.getT2()))
@@ -335,22 +316,25 @@ void subscriberContextIsNotMissing() {
final Mono mono1 = mono.contextWrite(reactorCtx -> reactorCtx.put("foo", "bar"));
StepVerifier.create(mono1)
- .expectNextMatches(s -> "baz".equals(s))
+ .expectNextMatches("baz"::equals)
.verifyComplete();
assertThat(ctxExists(ctx)).isFalse();
}
@Test
- void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread() {
+ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread()
+ throws InterruptedException {
// Given
final ClientRequestContext ctx = newContext();
final Mono mono;
+ final Scheduler single = Schedulers.single();
// When
mono = Mono.just("Hello")
+ .subscribeOn(single)
.delayElement(Duration.ofMillis(1000))
.map(s -> {
- if (s.equals("Hello")) {
+ if ("Hello".equals(s)) {
throw new RuntimeException();
}
return s;
@@ -362,29 +346,12 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
.expectError(RuntimeException.class)
.verify();
- assertThat(ctxExists(ctx)).isFalse();
- }
-
- @Test
- void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnMainThread() {
- // Given
- final ClientRequestContext ctx = newContext();
- final Mono mono;
-
- // When
- mono = Mono.just("Hello")
- .map(s -> {
- if (s.equals("Hello")) {
- throw new RuntimeException();
- }
- return s;
- })
- .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
-
- // Then
- StepVerifier.create(mono)
- .expectError(RuntimeException.class)
- .verify();
+ final CountDownLatch latch = new CountDownLatch(1);
+ single.schedule(() -> {
+ assertThat(ctxExists(ctx)).isFalse();
+ latch.countDown();
+ });
+ latch.await();
assertThat(ctxExists(ctx)).isFalse();
}
@@ -409,8 +376,7 @@ static ClientRequestContext newContext() {
}
private static Mono addCallbacks(Mono mono0, ClientRequestContext ctx,
- boolean shouldContextWrite,
- boolean shouldContextCapture) {
+ boolean useContextCapture) {
// doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
final Mono mono = mono0.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
@@ -420,15 +386,9 @@ private static Mono addCallbacks(Mono mono0, ClientRequestContext ctx,
.doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
.doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
-
- if (shouldContextWrite) {
- return mono.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
- }
-
- if (shouldContextCapture) {
+ if (useContextCapture) {
return mono.contextCapture();
}
-
- return mono;
+ return mono.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
}
}
From f23f67037e391f9ed5581ce8fe86d63cc7efbece Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Mon, 22 Apr 2024 16:29:53 +0900
Subject: [PATCH 25/48] add @UnstableAPI and remove whitespace.
---
.../com/linecorp/armeria/common/RequestContextAccessor.java | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index 1ac0d59892a..9d2728575af 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -17,6 +17,7 @@
import org.reactivestreams.Subscription;
+import com.linecorp.armeria.common.annotation.UnstableApi;
import com.linecorp.armeria.internal.common.RequestContextUtil;
import io.micrometer.context.ContextRegistry;
@@ -38,6 +39,7 @@
* Flux#contextWrite(ContextView) to end of the Reactor codes.
* If not, {@link RequestContext} will not be keep during Reactor Operation.
*/
+@UnstableApi
public final class RequestContextAccessor implements ThreadLocalAccessor {
private static final String KEY = RequestContextAccessor.class.getName();
@@ -58,7 +60,6 @@ public Object key() {
* User can use this method to register {@link RequestContext} to
* Reactor Context.
*/
-
public static String accessorKey() {
return KEY;
}
From eb0c71e9e68c77cef6780e0f18d4e5214dfab14a Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Tue, 23 Apr 2024 23:21:18 +0900
Subject: [PATCH 26/48] Update
core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
Co-authored-by: Trustin Lee
---
.../linecorp/armeria/common/RequestContextAccessorTest.java | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
diff --git a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
index 57a830c1844..7918a4620a5 100644
--- a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
@@ -145,7 +145,6 @@ void should_be_null_when_restore() {
}
static ClientRequestContext newContext() {
- return ClientRequestContext.builder(HttpRequest.of(HttpMethod.GET, "/"))
- .build();
+ return ClientRequestContext.of(HttpRequest.of(HttpMethod.GET, "/"));
}
}
From 8a46ca5506f3f6d2f328321618dbb9c3f71877fb Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Tue, 23 Apr 2024 23:40:25 +0900
Subject: [PATCH 27/48] apply review
---
core/build.gradle | 2 +-
.../common/RequestContextAccessor.java | 2 +-
.../common/RequestContextAccessorTest.java | 22 ++++---------------
dependencies.toml | 4 ++++
reactor3/build.gradle | 2 +-
.../RequestContextPropagationFluxTest.java | 2 +-
.../RequestContextPropagationMonoTest.java | 2 +-
7 files changed, 13 insertions(+), 23 deletions(-)
diff --git a/core/build.gradle b/core/build.gradle
index b0825a97d83..a4680f009dc 100644
--- a/core/build.gradle
+++ b/core/build.gradle
@@ -108,7 +108,7 @@ dependencies {
optionalApi libs.micrometer.prometheus
optionalApi libs.dropwizard.metrics.core
optionalApi libs.prometheus
- implementation 'io.micrometer:context-propagation:1.1.1'
+ api libs.context.propagation
// Netty
api libs.netty.transport
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
index 9d2728575af..b91fb72f537 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2020 LINE Corporation
+ * Copyright 2024 LINE Corporation
*
* LINE Corporation licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
diff --git a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
index 7918a4620a5..ae529da4210 100644
--- a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2019 LINE Corporation
+ * Copyright 2024 LINE Corporation
*
* LINE Corporation licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
@@ -13,24 +13,10 @@
* License for the specific language governing permissions and limitations
* under the License.
*/
-/*
- * Copyright 2012 The Netty Project
- *
- * The Netty Project licenses this file to you under the Apache License,
- * version 2.0 (the "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at:
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations
- * under the License.
- */
package com.linecorp.armeria.common;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import java.util.List;
@@ -90,8 +76,8 @@ void should_throw_NPE_when_set_null() {
final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
// When + Then
- Assertions.assertThatThrownBy(() -> reqCtxAccessor.setValue(null))
- .isInstanceOf(NullPointerException.class);
+ assertThatThrownBy(() -> reqCtxAccessor.setValue(null))
+ .isInstanceOf(NullPointerException.class);
}
@Test
diff --git a/dependencies.toml b/dependencies.toml
index ee8677f051d..6936187bf0a 100644
--- a/dependencies.toml
+++ b/dependencies.toml
@@ -18,6 +18,7 @@ caffeine = "2.9.3"
cglib = "3.3.0"
checkerframework = "2.5.6"
checkstyle = "10.3.2"
+context-propagation = "1.1.1"
controlplane = "1.0.42"
curator = "5.6.0"
dagger = "2.50"
@@ -979,6 +980,9 @@ javadocs = "https://developers.curioswitch.org/apidocs/java/"
module = "io.projectreactor:reactor-core"
version.ref = "reactor"
javadocs = "https://projectreactor.io/docs/core/release/api/"
+[libraries.context-propagation]
+module = "io.micrometer:context-propagation"
+version.ref = "context-propagation"
[libraries.reactor-test]
module = "io.projectreactor:reactor-test"
version.ref = "reactor"
diff --git a/reactor3/build.gradle b/reactor3/build.gradle
index e2b7194a7e6..c5e750d5057 100644
--- a/reactor3/build.gradle
+++ b/reactor3/build.gradle
@@ -1,4 +1,4 @@
dependencies {
api libs.reactor.core
- implementation 'io.micrometer:context-propagation:1.1.1'
+ api libs.context.propagation
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 7297117e3d6..67a24945de6 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2020 LINE Corporation
+ * Copyright 2024 LINE Corporation
*
* LINE Corporation licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 5cfc4f55a0e..fc5d39100c7 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2020 LINE Corporation
+ * Copyright 2024 LINE Corporation
*
* LINE Corporation licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
From 9cd60e1c8a9c6969e3a09fe21db345c258d716a5 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 24 Apr 2024 08:31:54 +0900
Subject: [PATCH 28/48] apply review
---
.../RequestContextThreadLocalAccessor.java} | 29 +++++++------------
.../io.micrometer.context.ThreadLocalAccessor | 2 +-
...equestContextThreadLocalAccessorTest.java} | 25 +++++++++-------
.../RequestContextPropagationFluxTest.java | 7 +++--
.../RequestContextPropagationMonoTest.java | 6 ++--
5 files changed, 32 insertions(+), 37 deletions(-)
rename core/src/main/java/com/linecorp/armeria/{common/RequestContextAccessor.java => internal/common/RequestContextThreadLocalAccessor.java} (84%)
rename core/src/test/java/com/linecorp/armeria/{common/RequestContextAccessorTest.java => internal/common/RequestContextThreadLocalAccessorTest.java} (77%)
diff --git a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
similarity index 84%
rename from core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
rename to core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
index b91fb72f537..38bd6001bd7 100644
--- a/core/src/main/java/com/linecorp/armeria/common/RequestContextAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
@@ -13,12 +13,13 @@
* License for the specific language governing permissions and limitations
* under the License.
*/
-package com.linecorp.armeria.common;
+package com.linecorp.armeria.internal.common;
import org.reactivestreams.Subscription;
+import com.linecorp.armeria.common.RequestContext;
+import com.linecorp.armeria.common.RequestContextStorage;
import com.linecorp.armeria.common.annotation.UnstableApi;
-import com.linecorp.armeria.internal.common.RequestContextUtil;
import io.micrometer.context.ContextRegistry;
import io.micrometer.context.ContextSnapshot;
@@ -30,8 +31,8 @@
*
* Context-propagation library and keep the {@link RequestContext} during
* Reactor operations.
- * Get the {@link RequestContextAccessor} to register it to the {@link ContextRegistry}.
- * Then, {@link ContextRegistry} will use {@link RequestContextAccessor} to
+ * Get the {@link RequestContextThreadLocalAccessor} to register it to the {@link ContextRegistry}.
+ * Then, {@link ContextRegistry} will use {@link RequestContextThreadLocalAccessor} to
* propagate context during the
* Reactor operations
* so that you can get the context using {@link RequestContext#current()}.
@@ -40,12 +41,12 @@
* If not, {@link RequestContext} will not be keep during Reactor Operation.
*/
@UnstableApi
-public final class RequestContextAccessor implements ThreadLocalAccessor {
+public final class RequestContextThreadLocalAccessor implements ThreadLocalAccessor {
- private static final String KEY = RequestContextAccessor.class.getName();
+ private static final Object KEY = RequestContext.class;
/**
- * The value which obtained through {@link RequestContextAccessor},
+ * The value which obtained through {@link RequestContextThreadLocalAccessor},
* will be stored in the Context under this {@code KEY}.
* This method will be called by {@link ContextSnapshot} internally.
*/
@@ -54,16 +55,6 @@ public Object key() {
return KEY;
}
- /**
- * The value which obtained through {@link RequestContextAccessor},
- * will be stored in the Context under this {@code KEY}.
- * User can use this method to register {@link RequestContext} to
- * Reactor Context.
- */
- public static String accessorKey() {
- return KEY;
- }
-
/**
* {@link ContextSnapshot} will call this method during the execution
* of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
@@ -98,7 +89,7 @@ public void setValue(RequestContext value) {
/**
* This method will be called at the start of {@link ContextSnapshot.Scope} and
* the end of {@link ContextSnapshot.Scope}. If reactor Context does not
- * contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
+ * contains {@link RequestContextThreadLocalAccessor#KEY}, {@link ContextSnapshot} will use
* this method to remove the value from {@link ThreadLocal}.
* Please note that {@link RequestContextUtil#pop()} return {@link AutoCloseable} instance,
* but it is not used in `Try with Resources` syntax. this is because {@link ContextSnapshot.Scope}
@@ -129,7 +120,7 @@ public void restore(RequestContext previousValue) {
/**
* This method will be called at the start of {@link ContextSnapshot.Scope} and
* the end of {@link ContextSnapshot.Scope}. If reactor Context does not
- * contains {@link RequestContextAccessor#KEY}, {@link ContextSnapshot} will use
+ * contains {@link RequestContextThreadLocalAccessor#KEY}, {@link ContextSnapshot} will use
* this method to remove the value from {@link ThreadLocal}.
* Please note that {@link RequestContextUtil#pop()} return {@link AutoCloseable} instance,
* but it is not used in `Try with Resources` syntax. this is because {@link ContextSnapshot.Scope}
diff --git a/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor b/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
index 408c5a5ca30..61702a0e756 100644
--- a/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
+++ b/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
@@ -1 +1 @@
-com.linecorp.armeria.common.RequestContextAccessor
+com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor
diff --git a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java b/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
similarity index 77%
rename from core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
rename to core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
index ae529da4210..befd58cbf18 100644
--- a/core/src/test/java/com/linecorp/armeria/common/RequestContextAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
@@ -13,23 +13,26 @@
* License for the specific language governing permissions and limitations
* under the License.
*/
-package com.linecorp.armeria.common;
+package com.linecorp.armeria.internal.common;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import java.util.List;
-import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import com.linecorp.armeria.client.ClientRequestContext;
+import com.linecorp.armeria.common.HttpMethod;
+import com.linecorp.armeria.common.HttpRequest;
+import com.linecorp.armeria.common.RequestContext;
+import com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor;
import com.linecorp.armeria.internal.common.RequestContextUtil;
import io.micrometer.context.ContextRegistry;
import io.micrometer.context.ThreadLocalAccessor;
-class RequestContextAccessorTest {
+class RequestContextThreadLocalAccessorTest {
@Test
void should_be_loaded_by_SPI() {
@@ -37,14 +40,14 @@ void should_be_loaded_by_SPI() {
final List> threadLocalAccessors = ctxRegistry.getThreadLocalAccessors();
assertThat(threadLocalAccessors.size()).isGreaterThan(1);
- assertThat(threadLocalAccessors).hasAtLeastOneElementOfType(RequestContextAccessor.class);
+ assertThat(threadLocalAccessors).hasAtLeastOneElementOfType(RequestContextThreadLocalAccessor.class);
}
@Test
void should_return_expected_key() {
// Given
- final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
- final String expectedValue = RequestContextAccessor.class.getName();
+ final RequestContextThreadLocalAccessor reqCtxAccessor = new RequestContextThreadLocalAccessor();
+ final Object expectedValue = RequestContext.class;
// When
final Object result = reqCtxAccessor.key();
@@ -58,7 +61,7 @@ void should_return_expected_key() {
void should_success_set() {
// Given
final ClientRequestContext ctx = newContext();
- final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final RequestContextThreadLocalAccessor reqCtxAccessor = new RequestContextThreadLocalAccessor();
// When
reqCtxAccessor.setValue(ctx);
@@ -73,7 +76,7 @@ void should_success_set() {
@Test
void should_throw_NPE_when_set_null() {
// Given
- final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final RequestContextThreadLocalAccessor reqCtxAccessor = new RequestContextThreadLocalAccessor();
// When + Then
assertThatThrownBy(() -> reqCtxAccessor.setValue(null))
@@ -84,7 +87,7 @@ void should_throw_NPE_when_set_null() {
void should_be_null_when_setValue() {
// Given
final ClientRequestContext ctx = newContext();
- final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final RequestContextThreadLocalAccessor reqCtxAccessor = new RequestContextThreadLocalAccessor();
reqCtxAccessor.setValue(ctx);
// When
@@ -99,7 +102,7 @@ void should_be_null_when_setValue() {
@SuppressWarnings("MustBeClosedChecker")
void should_be_restore_original_state_when_restore() {
// Given
- final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final RequestContextThreadLocalAccessor reqCtxAccessor = new RequestContextThreadLocalAccessor();
final ClientRequestContext previousCtx = newContext();
final ClientRequestContext currentCtx = newContext();
reqCtxAccessor.setValue(currentCtx);
@@ -118,7 +121,7 @@ void should_be_restore_original_state_when_restore() {
@Test
void should_be_null_when_restore() {
// Given
- final RequestContextAccessor reqCtxAccessor = new RequestContextAccessor();
+ final RequestContextThreadLocalAccessor reqCtxAccessor = new RequestContextThreadLocalAccessor();
final ClientRequestContext currentCtx = newContext();
reqCtxAccessor.setValue(currentCtx);
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 67a24945de6..84c1e40b139 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -34,7 +34,8 @@
import org.reactivestreams.Publisher;
import com.linecorp.armeria.client.ClientRequestContext;
-import com.linecorp.armeria.common.RequestContextAccessor;
+import com.linecorp.armeria.common.RequestContext;
+import com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor;
import com.linecorp.armeria.common.util.SafeCloseable;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
@@ -597,7 +598,7 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
}
return s;
})
- .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ .contextWrite(Context.of(RequestContext.class, ctx));
// Then
StepVerifier.create(flux)
@@ -630,6 +631,6 @@ private static Flux addCallbacks(Flux flux0,
if (useContextCapture) {
return flux.contextCapture();
}
- return flux.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ return flux.contextWrite(Context.of(RequestContext.class, ctx));
}
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index fc5d39100c7..3a0ed36e7f9 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -32,7 +32,7 @@
import com.linecorp.armeria.common.HttpMethod;
import com.linecorp.armeria.common.HttpRequest;
import com.linecorp.armeria.common.RequestContext;
-import com.linecorp.armeria.common.RequestContextAccessor;
+import com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor;
import com.linecorp.armeria.common.util.SafeCloseable;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
@@ -339,7 +339,7 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
}
return s;
})
- .contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ .contextWrite(Context.of(RequestContext.class, ctx));
// Then
StepVerifier.create(mono)
@@ -389,6 +389,6 @@ private static Mono addCallbacks(Mono mono0, ClientRequestContext ctx,
if (useContextCapture) {
return mono.contextCapture();
}
- return mono.contextWrite(Context.of(RequestContextAccessor.accessorKey(), ctx));
+ return mono.contextWrite(Context.of(RequestContext.class, ctx));
}
}
From 4b5015f8c8b77df69d2b43f8991fb5aa550adffa Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 24 Apr 2024 08:55:05 +0900
Subject: [PATCH 29/48] add test case
---
...RequestContextThreadLocalAccessorTest.java | 33 +++++++++++++++++--
1 file changed, 31 insertions(+), 2 deletions(-)
diff --git a/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java b/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
index befd58cbf18..1ba6c12f591 100644
--- a/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
@@ -26,10 +26,11 @@
import com.linecorp.armeria.common.HttpMethod;
import com.linecorp.armeria.common.HttpRequest;
import com.linecorp.armeria.common.RequestContext;
-import com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor;
-import com.linecorp.armeria.internal.common.RequestContextUtil;
import io.micrometer.context.ContextRegistry;
+import io.micrometer.context.ContextSnapshot;
+import io.micrometer.context.ContextSnapshot.Scope;
+import io.micrometer.context.ContextSnapshotFactory;
import io.micrometer.context.ThreadLocalAccessor;
class RequestContextThreadLocalAccessorTest {
@@ -133,6 +134,34 @@ void should_be_null_when_restore() {
assertThat(reqCtx).isNull();
}
+ @Test
+ void requestContext_should_exist_inside_scope_and_not_outside() {
+ // Given
+ final RequestContextThreadLocalAccessor reqCtxAccessor = new RequestContextThreadLocalAccessor();
+ final ClientRequestContext currentCtx = newContext();
+ final ClientRequestContext expectedCtx = currentCtx;
+ reqCtxAccessor.setValue(currentCtx);
+
+ final ContextSnapshotFactory factory = ContextSnapshotFactory.builder()
+ .clearMissing(true)
+ .build();
+ final ContextSnapshot contextSnapshot = factory.captureAll();
+ reqCtxAccessor.setValue();
+
+ // When : contextSnapshot.setThreadLocals()
+ try (Scope ignored = contextSnapshot.setThreadLocals()) {
+
+ // Then : should not
+ final RequestContext reqCtxInScope = RequestContext.currentOrNull();
+ assertThat(reqCtxInScope).isNotNull();
+ assertThat(reqCtxInScope).isEqualTo(expectedCtx);
+ }
+
+ // Then
+ final RequestContext reqCtxOutOfScope = RequestContext.currentOrNull();
+ assertThat(reqCtxOutOfScope).isNull();
+ }
+
static ClientRequestContext newContext() {
return ClientRequestContext.of(HttpRequest.of(HttpMethod.GET, "/"));
}
From 90f76a702df09cc795dc486d24532da16341f43b Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 24 Apr 2024 09:30:32 +0900
Subject: [PATCH 30/48] fix lint error
---
.../common/reactor3/RequestContextPropagationFluxTest.java | 1 -
.../common/reactor3/RequestContextPropagationMonoTest.java | 1 -
2 files changed, 2 deletions(-)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 84c1e40b139..5471b7d279d 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -35,7 +35,6 @@
import com.linecorp.armeria.client.ClientRequestContext;
import com.linecorp.armeria.common.RequestContext;
-import com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor;
import com.linecorp.armeria.common.util.SafeCloseable;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 3a0ed36e7f9..7a4d00e8298 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -32,7 +32,6 @@
import com.linecorp.armeria.common.HttpMethod;
import com.linecorp.armeria.common.HttpRequest;
import com.linecorp.armeria.common.RequestContext;
-import com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor;
import com.linecorp.armeria.common.util.SafeCloseable;
import com.linecorp.armeria.internal.testing.AnticipatedException;
import com.linecorp.armeria.internal.testing.GenerateNativeImageTrace;
From 230a664164ef4e41a532c16336257541f6da4deb Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Wed, 24 Apr 2024 12:25:08 +0900
Subject: [PATCH 31/48] Update
core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
Co-authored-by: Trustin Lee
---
.../internal/common/RequestContextThreadLocalAccessor.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
index 38bd6001bd7..7e07c6227ac 100644
--- a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
@@ -40,7 +40,6 @@
* Flux#contextWrite(ContextView) to end of the Reactor codes.
* If not, {@link RequestContext} will not be keep during Reactor Operation.
*/
-@UnstableApi
public final class RequestContextThreadLocalAccessor implements ThreadLocalAccessor {
private static final Object KEY = RequestContext.class;
From 3b9de29fc184c30a5a855e4c19e0a2b95e92c355 Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Wed, 24 Apr 2024 12:25:17 +0900
Subject: [PATCH 32/48] Update
core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
Co-authored-by: Trustin Lee
---
.../internal/common/RequestContextThreadLocalAccessorTest.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java b/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
index 1ba6c12f591..594f2e9d990 100644
--- a/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
+++ b/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
@@ -154,7 +154,7 @@ void requestContext_should_exist_inside_scope_and_not_outside() {
// Then : should not
final RequestContext reqCtxInScope = RequestContext.currentOrNull();
assertThat(reqCtxInScope).isNotNull();
- assertThat(reqCtxInScope).isEqualTo(expectedCtx);
+ assertThat(reqCtxInScope).isSameAs(expectedCtx);
}
// Then
From 890a7257586409ecf82e4b5fdda2e4b36021f38c Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 24 Apr 2024 12:45:36 +0900
Subject: [PATCH 33/48] apply review
---
core/build.gradle | 2 +-
.../internal/common/RequestContextThreadLocalAccessor.java | 1 -
dependencies.toml | 7 ++++---
reactor3/build.gradle | 2 +-
4 files changed, 6 insertions(+), 6 deletions(-)
diff --git a/core/build.gradle b/core/build.gradle
index 544fa79b5ec..efbee9a2a26 100644
--- a/core/build.gradle
+++ b/core/build.gradle
@@ -108,7 +108,7 @@ dependencies {
optionalApi libs.micrometer.prometheus
optionalApi libs.dropwizard.metrics.core
optionalApi libs.prometheus
- api libs.context.propagation
+ implementation libs.context.propagation
// Netty
api libs.netty.transport
diff --git a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
index 7e07c6227ac..6aeb8be65bd 100644
--- a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
@@ -19,7 +19,6 @@
import com.linecorp.armeria.common.RequestContext;
import com.linecorp.armeria.common.RequestContextStorage;
-import com.linecorp.armeria.common.annotation.UnstableApi;
import io.micrometer.context.ContextRegistry;
import io.micrometer.context.ContextSnapshot;
diff --git a/dependencies.toml b/dependencies.toml
index b28b8fb63fb..4a18d97f895 100644
--- a/dependencies.toml
+++ b/dependencies.toml
@@ -314,6 +314,10 @@ version.ref = "checkerframework"
module = "com.puppycrawl.tools:checkstyle"
version.ref = "checkstyle"
+[libraries.context-propagation]
+module = "io.micrometer:context-propagation"
+version.ref = "context-propagation"
+
[libraries.controlplane-api]
module = "io.envoyproxy.controlplane:api"
version.ref = "controlplane"
@@ -982,9 +986,6 @@ javadocs = "https://developers.curioswitch.org/apidocs/java/"
module = "io.projectreactor:reactor-core"
version.ref = "reactor"
javadocs = "https://projectreactor.io/docs/core/release/api/"
-[libraries.context-propagation]
-module = "io.micrometer:context-propagation"
-version.ref = "context-propagation"
[libraries.reactor-test]
module = "io.projectreactor:reactor-test"
version.ref = "reactor"
diff --git a/reactor3/build.gradle b/reactor3/build.gradle
index c5e750d5057..6a3ce0f183b 100644
--- a/reactor3/build.gradle
+++ b/reactor3/build.gradle
@@ -1,4 +1,4 @@
dependencies {
api libs.reactor.core
- api libs.context.propagation
+ implementation libs.context.propagation
}
From 6a3e46e99a2d19dffb5e25aa5872bbf64eead1ed Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Wed, 8 May 2024 15:13:56 +0900
Subject: [PATCH 34/48] Update
core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
Co-authored-by: Trustin Lee
---
.../internal/common/RequestContextThreadLocalAccessor.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
index 6aeb8be65bd..19678fad38b 100644
--- a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
@@ -27,8 +27,8 @@
/**
* This class works with the
- *
- * Context-propagation library and keep the {@link RequestContext} during
+ * Micrometer
+ * Context Propagation to keep the {@link RequestContext} during
* Reactor operations.
* Get the {@link RequestContextThreadLocalAccessor} to register it to the {@link ContextRegistry}.
* Then, {@link ContextRegistry} will use {@link RequestContextThreadLocalAccessor} to
From e80117ccc1945044ae8e3aac9b21771d45343833 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Wed, 8 May 2024 18:49:01 +0900
Subject: [PATCH 35/48] apply review
---
.../internal/common/RequestContextThreadLocalAccessor.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
index 19678fad38b..4096b20609e 100644
--- a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
@@ -81,7 +81,7 @@ public RequestContext getValue() {
@Override
@SuppressWarnings("MustBeClosedChecker")
public void setValue(RequestContext value) {
- value.push();
+ RequestContextUtil.getAndSet(value);
}
/**
@@ -112,7 +112,7 @@ public void setValue() {
@Override
@SuppressWarnings("MustBeClosedChecker")
public void restore(RequestContext previousValue) {
- previousValue.push();
+ RequestContextUtil.getAndSet(previousValue);
}
/**
From 79f16af9fcbded1fa366111c0f21cfa5abdcfe88 Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Thu, 20 Jun 2024 23:01:14 +0900
Subject: [PATCH 36/48] Update
reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
Co-authored-by: jrhee17
---
.../common/reactor3/RequestContextPropagationFluxTest.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 5471b7d279d..296549efd73 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -617,7 +617,6 @@ void ctxShouldBeCleanUpEvenIfErrorOccursDuringReactorOperationOnSchedulerThread(
private static Flux addCallbacks(Flux flux0,
ClientRequestContext ctx,
boolean useContextCapture) {
- // doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
final Flux flux = flux0.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnRequest(l -> assertThat(ctxExists(ctx)).isTrue())
From fb2db7dbae6a7f688ad32e47505eae552ca6b983 Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Thu, 20 Jun 2024 23:01:47 +0900
Subject: [PATCH 37/48] Update
reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
Co-authored-by: jrhee17
---
.../common/reactor3/RequestContextPropagationMonoTest.java | 1 -
1 file changed, 1 deletion(-)
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 7a4d00e8298..98026f4dc2e 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -376,7 +376,6 @@ static ClientRequestContext newContext() {
private static Mono addCallbacks(Mono mono0, ClientRequestContext ctx,
boolean useContextCapture) {
- // doOnCancel and doFinally do not have context because we cannot add a hook to the cancel.
final Mono mono = mono0.doFirst(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnSubscribe(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnRequest(l -> assertThat(ctxExists(ctx)).isTrue())
From f0879124d0dd64a6a2a103923012df945153e1d8 Mon Sep 17 00:00:00 2001
From: ChickenchickenLove
Date: Thu, 20 Jun 2024 23:07:12 +0900
Subject: [PATCH 38/48] Update
core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
Co-authored-by: jrhee17
---
.../RequestContextThreadLocalAccessor.java | 30 -------------------
1 file changed, 30 deletions(-)
diff --git a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
index 4096b20609e..1e7e21214e5 100644
--- a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
@@ -99,34 +99,4 @@ public void setValue() {
RequestContextUtil.pop();
}
- /**
- * {@link ContextSnapshot} will call this method during the execution
- * of lambda functions in {@link ContextSnapshot#wrap(Runnable)},
- * as well as during Mono#subscribe(), Flux#subscribe(),
- * {@link Subscription#request(long)}, and CoreSubscriber#onSubscribe(Subscription).
- * Following these calls, {@link ContextSnapshot#setThreadLocals()} is
- * invoked to restore the state of {@link RequestContextStorage}.
- * Furthermore, at the end of these methods, {@link Scope#close()} is executed
- * to revert the {@link RequestContextStorage} to its original state.
- */
- @Override
- @SuppressWarnings("MustBeClosedChecker")
- public void restore(RequestContext previousValue) {
- RequestContextUtil.getAndSet(previousValue);
- }
-
- /**
- * This method will be called at the start of {@link ContextSnapshot.Scope} and
- * the end of {@link ContextSnapshot.Scope}. If reactor Context does not
- * contains {@link RequestContextThreadLocalAccessor#KEY}, {@link ContextSnapshot} will use
- * this method to remove the value from {@link ThreadLocal}.
- * Please note that {@link RequestContextUtil#pop()} return {@link AutoCloseable} instance,
- * but it is not used in `Try with Resources` syntax. this is because {@link ContextSnapshot.Scope}
- * will handle the {@link AutoCloseable} instance returned by {@link RequestContextUtil#pop()}.
- */
- @Override
- @SuppressWarnings("MustBeClosedChecker")
- public void restore() {
- RequestContextUtil.pop();
- }
}
From 1e6cd9bb6e3c6d703b87a2ed592b8fd31cccd820 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Thu, 20 Jun 2024 23:17:10 +0900
Subject: [PATCH 39/48] Add validation callback for context-propagation.
---
.../internal/common/RequestContextThreadLocalAccessor.java | 1 -
.../common/reactor3/RequestContextPropagationFluxTest.java | 2 ++
.../common/reactor3/RequestContextPropagationMonoTest.java | 2 ++
3 files changed, 4 insertions(+), 1 deletion(-)
diff --git a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
index 1e7e21214e5..4b6a3903c70 100644
--- a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
+++ b/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
@@ -98,5 +98,4 @@ public void setValue(RequestContext value) {
public void setValue() {
RequestContextUtil.pop();
}
-
}
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
index 296549efd73..4221039b0c8 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationFluxTest.java
@@ -624,6 +624,8 @@ private static Flux addCallbacks(Flux flux0,
.doOnComplete(() -> assertThat(ctxExists(ctx)).isTrue())
.doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnCancel(() -> assertThat(ctxExists(ctx)).isTrue())
+ .doFinally(t -> assertThat(ctxExists(ctx)).isTrue())
.doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
if (useContextCapture) {
diff --git a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
index 98026f4dc2e..85366741982 100644
--- a/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
+++ b/reactor3/src/test/java/com/linecorp/armeria/common/reactor3/RequestContextPropagationMonoTest.java
@@ -383,6 +383,8 @@ private static Mono addCallbacks(Mono mono0, ClientRequestContext ctx,
.doOnSuccess(t -> assertThat(ctxExists(ctx)).isTrue())
.doOnEach(s -> assertThat(ctxExists(ctx)).isTrue())
.doOnError(t -> assertThat(ctxExists(ctx)).isTrue())
+ .doOnCancel(() -> assertThat(ctxExists(ctx)).isTrue())
+ .doFinally(t -> assertThat(ctxExists(ctx)).isTrue())
.doAfterTerminate(() -> assertThat(ctxExists(ctx)).isTrue());
if (useContextCapture) {
return mono.contextCapture();
From 640f904985f5faa4ef0f4b68042d2ef4db172e52 Mon Sep 17 00:00:00 2001
From: chickenchickenlove
Date: Sat, 26 Oct 2024 18:12:42 +0900
Subject: [PATCH 40/48] Create new module micrometer-context.
---
core/build.gradle | 1 -
.../io.micrometer.context.ThreadLocalAccessor | 1 -
micrometer-context/build.gradle | 3 +++
.../RequestContextThreadLocalAccessor.java | 3 ++-
.../RequestContextThreadLocalAccessorTest.java | 16 ++--------------
reactor3/build.gradle | 1 +
.../RequestContextPropagationFluxTest.java | 5 +++++
.../RequestContextPropagationMonoTest.java | 5 +++++
settings.gradle | 1 +
9 files changed, 19 insertions(+), 17 deletions(-)
delete mode 100644 core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
create mode 100644 micrometer-context/build.gradle
rename {core/src/main/java/com/linecorp/armeria/internal/common => micrometer-context/src/main/java/com/linecorp/armeria/common/micrometer/context}/RequestContextThreadLocalAccessor.java (97%)
rename {core/src/test/java/com/linecorp/armeria/internal/common => micrometer-context/src/test/java/com/linecorp/armeria/common/micrometer/context}/RequestContextThreadLocalAccessorTest.java (90%)
diff --git a/core/build.gradle b/core/build.gradle
index 1c83436b94f..3b4eed35289 100644
--- a/core/build.gradle
+++ b/core/build.gradle
@@ -110,7 +110,6 @@ dependencies {
optionalApi libs.micrometer.prometheus.legacy
optionalApi libs.dropwizard.metrics.core
optionalApi libs.prometheus.legacy
- implementation libs.context.propagation
// Netty
api libs.netty.transport
diff --git a/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor b/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
deleted file mode 100644
index 61702a0e756..00000000000
--- a/core/src/main/resources/META-INF/services/io.micrometer.context.ThreadLocalAccessor
+++ /dev/null
@@ -1 +0,0 @@
-com.linecorp.armeria.internal.common.RequestContextThreadLocalAccessor
diff --git a/micrometer-context/build.gradle b/micrometer-context/build.gradle
new file mode 100644
index 00000000000..ad450aebdce
--- /dev/null
+++ b/micrometer-context/build.gradle
@@ -0,0 +1,3 @@
+dependencies {
+ implementation libs.context.propagation
+}
diff --git a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java b/micrometer-context/src/main/java/com/linecorp/armeria/common/micrometer/context/RequestContextThreadLocalAccessor.java
similarity index 97%
rename from core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
rename to micrometer-context/src/main/java/com/linecorp/armeria/common/micrometer/context/RequestContextThreadLocalAccessor.java
index 4b6a3903c70..7bde73b0b3b 100644
--- a/core/src/main/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessor.java
+++ b/micrometer-context/src/main/java/com/linecorp/armeria/common/micrometer/context/RequestContextThreadLocalAccessor.java
@@ -13,12 +13,13 @@
* License for the specific language governing permissions and limitations
* under the License.
*/
-package com.linecorp.armeria.internal.common;
+package com.linecorp.armeria.common.micrometer.context;
import org.reactivestreams.Subscription;
import com.linecorp.armeria.common.RequestContext;
import com.linecorp.armeria.common.RequestContextStorage;
+import com.linecorp.armeria.internal.common.RequestContextUtil;
import io.micrometer.context.ContextRegistry;
import io.micrometer.context.ContextSnapshot;
diff --git a/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java b/micrometer-context/src/test/java/com/linecorp/armeria/common/micrometer/context/RequestContextThreadLocalAccessorTest.java
similarity index 90%
rename from core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
rename to micrometer-context/src/test/java/com/linecorp/armeria/common/micrometer/context/RequestContextThreadLocalAccessorTest.java
index 594f2e9d990..3ff4530a04f 100644
--- a/core/src/test/java/com/linecorp/armeria/internal/common/RequestContextThreadLocalAccessorTest.java
+++ b/micrometer-context/src/test/java/com/linecorp/armeria/common/micrometer/context/RequestContextThreadLocalAccessorTest.java
@@ -13,37 +13,25 @@
* License for the specific language governing permissions and limitations
* under the License.
*/
-package com.linecorp.armeria.internal.common;
+package com.linecorp.armeria.common.micrometer.context;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import java.util.List;
-
import org.junit.jupiter.api.Test;
import com.linecorp.armeria.client.ClientRequestContext;
import com.linecorp.armeria.common.HttpMethod;
import com.linecorp.armeria.common.HttpRequest;
import com.linecorp.armeria.common.RequestContext;
+import com.linecorp.armeria.internal.common.RequestContextUtil;
-import io.micrometer.context.ContextRegistry;
import io.micrometer.context.ContextSnapshot;
import io.micrometer.context.ContextSnapshot.Scope;
import io.micrometer.context.ContextSnapshotFactory;
-import io.micrometer.context.ThreadLocalAccessor;
class RequestContextThreadLocalAccessorTest {
- @Test
- void should_be_loaded_by_SPI() {
- final ContextRegistry ctxRegistry = ContextRegistry.getInstance();
- final List