From 899abf76e7a381abdfbc30f08a6a5678c0a8065d Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Wed, 19 Feb 2025 14:28:33 -0800 Subject: [PATCH 1/9] Backport fc98998627443d6e73ac70661f47f48b30525712 --- .../jdk/internal/net/http/HttpClientImpl.java | 35 +- .../jdk/internal/net/http/MultiExchange.java | 39 +- .../httpclient/HttpGetInCancelledFuture.java | 391 ++++++++++++++++++ .../java/net/httpclient/ReferenceTracker.java | 12 +- 4 files changed, 453 insertions(+), 24 deletions(-) create mode 100644 test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java b/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java index eea88945610..d88923192e5 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java @@ -282,23 +282,25 @@ public String toString() { } } - static void registerPending(PendingRequest pending) { + static CompletableFuture registerPending(PendingRequest pending, CompletableFuture res) { // shortcut if cf is already completed: no need to go through the trouble of // registering it - if (pending.cf.isDone()) return; + if (pending.cf.isDone()) return res; var client = pending.client; var cf = pending.cf; var id = pending.id; boolean added = client.pendingRequests.add(pending); // this may immediately remove `pending` from the set is the cf is already completed - pending.ref = cf.whenComplete((r,t) -> client.pendingRequests.remove(pending)); + var ref = res.whenComplete((r,t) -> client.pendingRequests.remove(pending)); + pending.ref = ref; assert added : "request %d was already added".formatted(id); // should not happen, unless the selector manager has already // exited abnormally if (client.selmgr.isClosed()) { pending.abort(client.selmgr.selectorClosedException()); } + return ref; } static void abortPendingRequests(HttpClientImpl client, Throwable reason) { @@ -930,8 +932,9 @@ private void debugCompleted(String tag, long startNanos, HttpRequest req) { cf = sendAsync(req, responseHandler, null, null); return cf.get(); } catch (InterruptedException ie) { - if (cf != null ) + if (cf != null) { cf.cancel(true); + } throw ie; } catch (ExecutionException e) { final Throwable throwable = e.getCause(); @@ -1053,19 +1056,23 @@ private void debugCompleted(String tag, long startNanos, HttpRequest req) { (b,t) -> debugCompleted("ClientImpl (async)", start, userRequest)); } - // makes sure that any dependent actions happen in the CF default - // executor. This is only needed for sendAsync(...), when - // exchangeExecutor is non-null. - if (exchangeExecutor != null) { - res = res.whenCompleteAsync((r, t) -> { /* do nothing */}, ASYNC_POOL); - } - // The mexCf is the Cf we need to abort if the SelectorManager thread // is aborted. PendingRequest pending = new PendingRequest(id, requestImpl, mexCf, mex, this); - registerPending(pending); - return res; - } catch(Throwable t) { + res = registerPending(pending, res); + + if (exchangeExecutor != null) { + // makes sure that any dependent actions happen in the CF default + // executor. This is only needed for sendAsync(...), when + // exchangeExecutor is non-null. + return res.isDone() ? res + : res.whenCompleteAsync((r, t) -> { /* do nothing */}, ASYNC_POOL); + } else { + // make a defensive copy that can be safely canceled + // by the caller + return res.isDone() ? res : res.copy(); + } + } catch (Throwable t) { requestUnreference(); debugCompleted("ClientImpl (async)", start, userRequest); throw t; diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java index 3d1f9255565..b3bc83e93de 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java @@ -91,7 +91,7 @@ class MultiExchange implements Cancelable { Exchange previous; volatile Throwable retryCause; volatile boolean expiredOnce; - volatile HttpResponse response = null; + volatile HttpResponse response; // Maximum number of times a request will be retried/redirected // for any reason @@ -279,11 +279,19 @@ public void cancel(IOException cause) { @Override public boolean cancel(boolean mayInterruptIfRunning) { boolean cancelled = this.cancelled; + boolean firstCancel = false; if (!cancelled && mayInterruptIfRunning) { if (interrupted.get() == null) { - interrupted.compareAndSet(null, + firstCancel = interrupted.compareAndSet(null, new CancellationException("Request cancelled")); } + if (debug.on()) { + if (firstCancel) { + debug.log("multi exchange recording: " + interrupted.get()); + } else { + debug.log("multi exchange recorded: " + interrupted.get()); + } + } this.cancelled = true; var exchange = getExchange(); if (exchange != null) { @@ -365,17 +373,30 @@ private CompletableFuture> handleNoBody(Response r, Exchange }).exceptionallyCompose(this::whenCancelled); } + // returns a CancellationExcpetion that wraps the given cause + // if cancel(boolean) was called, the given cause otherwise + private Throwable wrapIfCancelled(Throwable cause) { + CancellationException interrupt = interrupted.get(); + if (interrupt == null) return cause; + + var cancel = new CancellationException(interrupt.getMessage()); + // preserve the stack trace of the original exception to + // show where the call to cancel(boolean) came from + cancel.setStackTrace(interrupt.getStackTrace()); + cancel.initCause(Utils.getCancelCause(cause)); + return cancel; + } + + // if the request failed because the multi exchange was cancelled, + // make sure the reported exception is wrapped in CancellationException private CompletableFuture> whenCancelled(Throwable t) { - CancellationException x = interrupted.get(); - if (x != null) { - // make sure to fail with CancellationException if cancel(true) - // was called. - t = x.initCause(Utils.getCancelCause(t)); + var x = wrapIfCancelled(t); + if (x instanceof CancellationException) { if (debug.on()) { - debug.log("MultiExchange interrupted with: " + t.getCause()); + debug.log("MultiExchange interrupted with: " + x.getCause()); } } - return MinimalFuture.failedFuture(t); + return MinimalFuture.failedFuture(x); } static class NullSubscription implements Flow.Subscription { diff --git a/test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java b/test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java new file mode 100644 index 00000000000..5efc50138d6 --- /dev/null +++ b/test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java @@ -0,0 +1,391 @@ +/* + * Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved. + * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. + * + * This code is free software; you can redistribute it and/or modify it + * under the terms of the GNU General Public License version 2 only, as + * published by the Free Software Foundation. + * + * This code is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License + * version 2 for more details (a copy is included in the LICENSE file that + * accompanied this code). + * + * You should have received a copy of the GNU General Public License version + * 2 along with this work; if not, write to the Free Software Foundation, + * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. + * + * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA + * or visit www.oracle.com if you need additional information or have any + * questions. + */ + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpClient.Version; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import jdk.internal.net.http.common.OperationTrackers.Tracker; +import jdk.test.lib.net.SimpleSSLContext; +import jdk.test.lib.net.URIBuilder; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/* + * @test + * @bug 8316580 + * @library /test/lib + * @run junit/othervm -Djdk.tracePinnedThreads=full + * -DuseReferenceTracker=false + * HttpGetInCancelledFuture + * @run junit/othervm -Djdk.tracePinnedThreads=full + * -DuseReferenceTracker=true + * HttpGetInCancelledFuture + * @summary This test verifies that cancelling a future that + * does an HTTP request using the HttpClient doesn't cause + * HttpClient::close to block forever. + */ +public class HttpGetInCancelledFuture { + + static final boolean useTracker = Boolean.getBoolean("useReferenceTracker"); + + static final class TestException extends RuntimeException { + public TestException(String message, Throwable cause) { + super(message, cause); + } + } + + static ReferenceTracker TRACKER = ReferenceTracker.INSTANCE; + + HttpClient makeClient(URI uri, Version version, Executor executor) { + var builder = HttpClient.newBuilder(); + if (uri.getScheme().equalsIgnoreCase("https")) { + try { + builder.sslContext(new SimpleSSLContext().get()); + } catch (IOException io) { + throw new UncheckedIOException(io); + } + } + return builder.connectTimeout(Duration.ofSeconds(1)) + .executor(executor) + .version(version) + .build(); + } + + record TestCase(String url, int reqCount, Version version) {} + // A server that doesn't accept + static volatile ServerSocket NOT_ACCEPTING; + + static List parameters() { + ServerSocket ss = NOT_ACCEPTING; + if (ss == null) { + synchronized (HttpGetInCancelledFuture.class) { + if ((ss = NOT_ACCEPTING) == null) { + try { + ss = new ServerSocket(); + var loopback = InetAddress.getLoopbackAddress(); + ss.bind(new InetSocketAddress(loopback, 0), 10); + NOT_ACCEPTING = ss; + } catch (IOException io) { + throw new UncheckedIOException(io); + } + } + } + } + URI http = URIBuilder.newBuilder() + .loopback() + .scheme("http") + .port(ss.getLocalPort()) + .path("/not-accepting/") + .buildUnchecked(); + URI https = URIBuilder.newBuilder() + .loopback() + .scheme("https") + .port(ss.getLocalPort()) + .path("/not-accepting/") + .buildUnchecked(); + // use all HTTP versions, without and with TLS + return List.of( + new TestCase(http.toString(), 200, Version.HTTP_2), + new TestCase(http.toString(), 200, Version.HTTP_1_1), + new TestCase(https.toString(), 200, Version.HTTP_2), + new TestCase(https.toString(), 200, Version.HTTP_1_1) + ); + } + + @ParameterizedTest + @MethodSource("parameters") + void runTest(TestCase test) { + System.out.println("Testing with: " + test); + runTest(test.url, test.reqCount, test.version); + } + + static class TestTaskScope implements AutoCloseable { + final ExecutorService pool = new ForkJoinPool(); + final Map, Future> tasks = new ConcurrentHashMap<>(); + final AtomicReference failed = new AtomicReference<>(); + + class Task implements Callable { + final Callable task; + final CompletableFuture cf = new CompletableFuture<>(); + Task(Callable task) { + this.task = task; + } + @Override + public T call() throws Exception { + try { + var res = task.call(); + cf.complete(res); + return res; + } catch (Throwable t) { + cf.completeExceptionally(t); + throw t; + } + } + CompletableFuture cf() { + return cf; + } + } + + + static class ShutdownOnFailure extends TestTaskScope { + public ShutdownOnFailure() {} + + @Override + protected void completed(Task task, T result, Throwable throwable) { + super.completed(task, result, throwable); + if (throwable != null) { + if (failed.get() == null) { + ExecutionException ex = throwable instanceof ExecutionException x + ? x : new ExecutionException(throwable); + failed.compareAndSet(null, ex); + } + tasks.entrySet().forEach(this::cancel); + } + } + + void cancel(Map.Entry, Future> entry) { + entry.getValue().cancel(true); + entry.getKey().cf().cancel(true); + tasks.remove(entry.getKey(), entry.getValue()); + } + + @Override + public CompletableFuture fork(Callable callable) { + var ex = failed.get(); + if (ex == null) { + return super.fork(callable); + } // otherwise do nothing + return CompletableFuture.failedFuture(new RejectedExecutionException()); + } + } + + public CompletableFuture fork(Callable callable) { + var task = new Task<>(callable); + var res = pool.submit(task); + tasks.put(task, res); + task.cf.whenComplete((r,t) -> completed(task, r, t)); + return task.cf; + } + + protected void completed(Task task, T result, Throwable throwable) { + tasks.remove(task); + } + + public void join() throws InterruptedException { + try { + var cfs = tasks.keySet().stream() + .map(Task::cf).toArray(CompletableFuture[]::new); + CompletableFuture.allOf(cfs).get(); + } catch (InterruptedException it) { + throw it; + } catch (ExecutionException ex) { + failed.compareAndSet(null, ex); + } + } + + public void throwIfFailed() throws ExecutionException { + ExecutionException x = failed.get(); + if (x != null) throw x; + } + + public void close() { + pool.close(); + } + } + + ExecutorService testExecutor() { + return Executors.newCachedThreadPool(); + } + + void runTest(String url, int reqCount, Version version) { + final var dest = URI.create(url); + try (final var executor = testExecutor()) { + var httpClient = makeClient(dest, version, executor); + TRACKER.track(httpClient); + Tracker tracker = TRACKER.getTracker(httpClient); + Throwable failed = null; + try { + try (final var scope = new TestTaskScope.ShutdownOnFailure()) { + launchAndProcessRequests(scope, httpClient, reqCount, dest); + } finally { + System.out.printf("StructuredTaskScope closed: STARTED=%s, SUCCESS=%s, INTERRUPT=%s, FAILED=%s%n", + STARTED.get(), SUCCESS.get(), INTERRUPT.get(), FAILED.get()); + } + System.out.println("ERROR: Expected TestException not thrown"); + throw new AssertionError("Expected TestException not thrown"); + } catch (TestException x) { + System.out.println("Got expected exception: " + x); + } catch (Throwable t) { + System.out.println("ERROR: Unexpected exception: " + t); + failed = t; + throw t; + } finally { + // we can either use the tracker or call HttpClient::close + if (useTracker) { + // using the tracker depends on GC but will give us some diagnostic + // if some operations are not properly cancelled and prevent the client + // from terminating + httpClient = null; + System.gc(); + System.out.println(TRACKER.diagnose(tracker)); + var error = TRACKER.check(tracker, 10000); + if (error != null) { + if (failed != null) error.addSuppressed(failed); + EXCEPTIONS.forEach(x -> { + System.out.println("FAILED: " + x); + }); + EXCEPTIONS.forEach(x -> { + x.printStackTrace(System.out); + }); + throw error; + } + } else { + // if not all operations terminate, close() will block + // forever and the test will fail in jtreg timeout. + // there will be no diagnostic. + httpClient.close(); + } + System.out.println("HttpClient closed"); + } + } finally { + System.out.println("ThreadExecutor closed"); + } + // not all tasks may have been started before the scope was cancelled + // due to the first connect/timeout exception, but all tasks that started + // must have either succeeded, be interrupted, or failed + assertTrue(STARTED.get() > 0); + assertEquals(STARTED.get(), SUCCESS.get() + INTERRUPT.get() + FAILED.get()); + if (SUCCESS.get() > 0) { + // we don't expect any server to be listening and responding + System.out.println("WARNING: got some unexpected successful responses from " + + "\"" + NOT_ACCEPTING.getLocalSocketAddress() + "\": " + SUCCESS.get()); + } + } + + private void launchAndProcessRequests( + TestTaskScope.ShutdownOnFailure scope, + HttpClient httpClient, + int reqCount, + URI dest) { + for (int counter = 0; counter < reqCount; counter++) { + scope.fork(() -> + getAndCheck(httpClient, dest) + ); + } + try { + scope.join(); + } catch (InterruptedException e) { + throw new AssertionError("scope.join() was interrupted", e); + } + try { + scope.throwIfFailed(); + } catch (ExecutionException e) { + throw new TestException("something threw an exception in StructuredTaskScope", e); + } + } + + final static AtomicLong ID = new AtomicLong(); + final AtomicLong SUCCESS = new AtomicLong(); + final AtomicLong INTERRUPT = new AtomicLong(); + final AtomicLong FAILED = new AtomicLong(); + final AtomicLong STARTED = new AtomicLong(); + final CopyOnWriteArrayList EXCEPTIONS = new CopyOnWriteArrayList<>(); + private String getAndCheck(HttpClient httpClient, URI url) { + STARTED.incrementAndGet(); + final var response = sendRequest(httpClient, url); + String res = response.body(); + int statusCode = response.statusCode(); + assertEquals(200, statusCode); + return res; + } + + private HttpResponse sendRequest(HttpClient httpClient, URI url) { + var id = ID.incrementAndGet(); + try { + var request = HttpRequest.newBuilder(url).GET().build(); + var response = httpClient.send(request, HttpResponse.BodyHandlers.ofString()); + // System.out.println("Got response for " + id + ": " + response); + SUCCESS.incrementAndGet(); + return response; + } catch (InterruptedException e) { + INTERRUPT.incrementAndGet(); + // System.out.println("Got interrupted for " + id + ": " + e); + throw new RuntimeException(e); + } catch (Exception e) { + FAILED.incrementAndGet(); + EXCEPTIONS.add(e); + //System.out.println("Got exception for " + id + ": " + e); + throw new RuntimeException(e); + } + } + + @AfterAll + static void tearDown() { + try { + System.gc(); + var error = TRACKER.check(5000); + if (error != null) throw error; + } finally { + ServerSocket ss; + synchronized (HttpGetInCancelledFuture.class) { + ss = NOT_ACCEPTING; + NOT_ACCEPTING = null; + } + if (ss != null) { + try { + ss.close(); + } catch (IOException io) { + throw new UncheckedIOException(io); + } + } + } + } +} + diff --git a/test/jdk/java/net/httpclient/ReferenceTracker.java b/test/jdk/java/net/httpclient/ReferenceTracker.java index 42a7b0cf77a..dbc4a128675 100644 --- a/test/jdk/java/net/httpclient/ReferenceTracker.java +++ b/test/jdk/java/net/httpclient/ReferenceTracker.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018, 2022, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2018, 2023, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -246,6 +246,11 @@ public AssertionError check(Tracker tracker, } long duration = Duration.ofNanos(System.nanoTime() - waitStart).toMillis(); if (hasOutstanding.test(tracker)) { + if (i == 0 && waited == 0) { + // we found nothing and didn't wait expecting success, but then found + // something. Respin to make sure we wait. + return check(tracker, graceDelayMs, hasOutstanding, description, printThreads); + } StringBuilder warnings = diagnose(tracker, new StringBuilder(), hasOutstanding); if (hasOutstanding.test(tracker)) { fail = new AssertionError(warnings.toString()); @@ -302,6 +307,11 @@ public AssertionError check(long graceDelayMs, } long duration = Duration.ofNanos(System.nanoTime() - waitStart).toMillis(); if (TRACKERS.stream().anyMatch(hasOutstanding)) { + if (i == 0 && waited == 0) { + // we found nothing and didn't wait expecting success, but then found + // something. Respin to make sure we wait. + return check(graceDelayMs, hasOutstanding, description, printThreads); + } StringBuilder warnings = diagnose(new StringBuilder(), hasOutstanding); addSummary(warnings); if (TRACKERS.stream().anyMatch(hasOutstanding)) { From a5b7bb4d432c868d598c35f54306e40747071ad7 Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Wed, 19 Feb 2025 15:50:39 -0800 Subject: [PATCH 2/9] Backport 720b44648bcff997278af92746f942b2425298a5 --- .../jdk/internal/net/http/ExchangeImpl.java | 15 +- .../internal/net/http/Http2Connection.java | 55 ++- .../jdk/internal/net/http/MultiExchange.java | 28 +- .../classes/jdk/internal/net/http/Stream.java | 67 +++- .../internal/net/http/WindowController.java | 13 +- .../internal/net/http/frame/GoAwayFrame.java | 6 +- .../net/httpclient/http2/H2GoAwayTest.java | 336 ++++++++++++++++++ .../test/lib/common/HttpServerAdapters.java | 30 +- .../test/lib/http2/Http2TestExchange.java | 8 +- .../test/lib/http2/Http2TestExchangeImpl.java | 5 + .../test/lib/http2/Http2TestServer.java | 14 +- .../lib/http2/Http2TestServerConnection.java | 100 +++++- 12 files changed, 623 insertions(+), 54 deletions(-) create mode 100644 test/jdk/java/net/httpclient/http2/H2GoAwayTest.java diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java b/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java index 852742b6606..71e669b7ce1 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2022, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -58,6 +58,10 @@ abstract class ExchangeImpl { final Exchange exchange; + // this will be set to true only when the peer explicitly states (through a GOAWAY frame or + // a relevant error code in reset frame) that the corresponding stream (id) wasn't processed + private volatile boolean unprocessedByPeer; + ExchangeImpl(Exchange e) { // e == null means a http/2 pushed stream this.exchange = e; @@ -264,4 +268,13 @@ void upgraded() { } // Called when server returns non 100 response to // an Expect-Continue void expectContinueFailed(int rcode) { } + + final boolean isUnprocessedByPeer() { + return this.unprocessedByPeer; + } + + // Marks the exchange as unprocessed by the peer + final void markUnprocessedByPeer() { + this.unprocessedByPeer = true; + } } diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java b/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java index 84c09d6bca6..bb7a17a89cb 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java @@ -49,6 +49,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Flow; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -396,6 +397,7 @@ private record PushContinuationState(PushPromiseDecoder pushContDecoder, PushPro private final String key; // for HttpClientImpl.connections map private final FramesDecoder framesDecoder; private final FramesEncoder framesEncoder = new FramesEncoder(); + private final AtomicLong lastProcessedStreamInGoAway = new AtomicLong(-1); /** * Send Window controller for both connection and stream windows. @@ -802,7 +804,9 @@ final int maxConcurrentServerInitiatedStreams() { void close() { if (markHalfClosedLocal()) { - if (connection.channel().isOpen()) { + // we send a GOAWAY frame only if the remote side hasn't already indicated + // the intention to close the connection by previously sending a GOAWAY of its own + if (connection.channel().isOpen() && !isMarked(closedState, HALF_CLOSED_REMOTE)) { Log.logTrace("Closing HTTP/2 connection: to {0}", connection.address()); GoAwayFrame f = new GoAwayFrame(0, ErrorFrame.NO_ERROR, @@ -1354,13 +1358,46 @@ private void handlePing(PingFrame frame) sendUnorderedFrame(frame); } - private void handleGoAway(GoAwayFrame frame) - throws IOException - { - if (markHalfClosedLRemote()) { - shutdown(new IOException( - connection.channel().getLocalAddress() - + ": GOAWAY received")); + private void handleGoAway(final GoAwayFrame frame) { + final long lastProcessedStream = frame.getLastStream(); + assert lastProcessedStream >= 0 : "unexpected last stream id: " + + lastProcessedStream + " in GOAWAY frame"; + + markHalfClosedRemote(); + setFinalStream(); // don't allow any new streams on this connection + if (debug.on()) { + debug.log("processing incoming GOAWAY with last processed stream id:%s in frame %s", + lastProcessedStream, frame); + } + // see if this connection has previously received a GOAWAY from the peer and if yes + // then check if this new last processed stream id is lesser than the previous + // known last processed stream id. Only update the last processed stream id if the new + // one is lesser than the previous one. + long prevLastProcessed = lastProcessedStreamInGoAway.get(); + while (prevLastProcessed == -1 || lastProcessedStream < prevLastProcessed) { + if (lastProcessedStreamInGoAway.compareAndSet(prevLastProcessed, + lastProcessedStream)) { + break; + } + prevLastProcessed = lastProcessedStreamInGoAway.get(); + } + handlePeerUnprocessedStreams(lastProcessedStreamInGoAway.get()); + } + + private void handlePeerUnprocessedStreams(final long lastProcessedStream) { + final AtomicInteger numClosed = new AtomicInteger(); // atomic merely to allow usage within lambda + streams.forEach((id, exchange) -> { + if (id > lastProcessedStream) { + // any streams with an stream id higher than the last processed stream + // can be retried (on a new connection). we close the exchange as unprocessed + // to facilitate the retrying. + client2.client().theExecutor().ensureExecutedAsync(exchange::closeAsUnprocessed); + numClosed.incrementAndGet(); + } + }); + if (debug.on()) { + debug.log(numClosed.get() + " stream(s), with id greater than " + lastProcessedStream + + ", will be closed as unprocessed"); } } @@ -1911,7 +1948,7 @@ private boolean markHalfClosedLocal() { return markClosedState(HALF_CLOSED_LOCAL); } - private boolean markHalfClosedLRemote() { + private boolean markHalfClosedRemote() { return markClosedState(HALF_CLOSED_REMOTE); } diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java index b3bc83e93de..dbbcb7da19c 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2023, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -90,7 +90,7 @@ class MultiExchange implements Cancelable { Exchange exchange; // the current exchange Exchange previous; volatile Throwable retryCause; - volatile boolean expiredOnce; + volatile boolean retriedOnce; volatile HttpResponse response; // Maximum number of times a request will be retried/redirected @@ -459,7 +459,7 @@ private CompletableFuture responseAsyncImpl() { return exch.ignoreBody().handle((r,t) -> { previousreq = currentreq; currentreq = newrequest; - expiredOnce = false; + retriedOnce = false; setExchange(new Exchange<>(currentreq, this, acc)); return responseAsyncImpl(); }).thenCompose(Function.identity()); @@ -472,7 +472,7 @@ private CompletableFuture responseAsyncImpl() { return completedFuture(response); } // all exceptions thrown are handled here - CompletableFuture errorCF = getExceptionalCF(ex); + CompletableFuture errorCF = getExceptionalCF(ex, exch.exchImpl); if (errorCF == null) { return responseAsyncImpl(); } else { @@ -544,34 +544,38 @@ private Throwable retryCause(Throwable t) { * Takes a Throwable and returns a suitable CompletableFuture that is * completed exceptionally, or null. */ - private CompletableFuture getExceptionalCF(Throwable t) { + private CompletableFuture getExceptionalCF(Throwable t, ExchangeImpl exchImpl) { if ((t instanceof CompletionException) || (t instanceof ExecutionException)) { if (t.getCause() != null) { t = t.getCause(); } } + final boolean retryAsUnprocessed = exchImpl != null && exchImpl.isUnprocessedByPeer(); if (cancelled && !requestCancelled() && t instanceof IOException) { if (!(t instanceof HttpTimeoutException)) { t = toTimeoutException((IOException)t); } - } else if (retryOnFailure(t)) { + } else if (retryAsUnprocessed || retryOnFailure(t)) { Throwable cause = retryCause(t); if (!(t instanceof ConnectException)) { // we may need to start a new connection, and if so // we want to start with a fresh connect timeout again. if (connectTimeout != null) connectTimeout.reset(); - if (!canRetryRequest(currentreq)) { - return failedFuture(cause); // fails with original cause + if (!retryAsUnprocessed && !canRetryRequest(currentreq)) { + // a (peer) processed request which cannot be retried, fail with + // the original cause + return failedFuture(cause); } } // ConnectException: retry, but don't reset the connectTimeout. // allow the retry mechanism to do its work retryCause = cause; - if (!expiredOnce) { + if (!retriedOnce) { if (debug.on()) - debug.log(t.getClass().getSimpleName() + " (async): retrying...", t); - expiredOnce = true; + debug.log(t.getClass().getSimpleName() + + " (async): retrying " + currentreq + " due to: ", t); + retriedOnce = true; // The connection was abruptly closed. // We return null to retry the same request a second time. // The request filters have already been applied to the @@ -582,7 +586,7 @@ private CompletableFuture getExceptionalCF(Throwable t) { } else { if (debug.on()) { debug.log(t.getClass().getSimpleName() - + " (async): already retried once.", t); + + " (async): already retried once " + currentreq, t); } t = cause; } diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java b/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java index d55a9a4b446..5ae9f707d7c 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java @@ -672,20 +672,39 @@ void handleReset(ResetFrame frame, Flow.Subscriber subscriber) { stateLock.unlock(); } try { - int error = frame.getErrorCode(); - IOException e = new IOException("Received RST_STREAM: " - + ErrorFrame.stringForCode(error)); - if (errorRef.compareAndSet(null, e)) { - if (subscriber != null) { - subscriber.onError(e); + final int error = frame.getErrorCode(); + // A REFUSED_STREAM error code implies that the stream wasn't processed by the + // peer and the client is free to retry the request afresh. + if (error == ErrorFrame.REFUSED_STREAM) { + // Here we arrange for the request to be retried. Note that we don't call + // closeAsUnprocessed() method here because the "closed" state is already set + // to true a few lines above and calling close() from within + // closeAsUnprocessed() will end up being a no-op. We instead do the additional + // bookkeeping here. + markUnprocessedByPeer(); + errorRef.compareAndSet(null, new IOException("request not processed by peer")); + if (debug.on()) { + debug.log("request unprocessed by peer (REFUSED_STREAM) " + this.request); + } + } else { + final String reason = ErrorFrame.stringForCode(error); + final IOException failureCause = new IOException("Received RST_STREAM: " + reason); + if (debug.on()) { + debug.log(streamid + " received RST_STREAM with code: " + reason); + } + if (errorRef.compareAndSet(null, failureCause)) { + if (subscriber != null) { + subscriber.onError(failureCause); + } } } - completeResponseExceptionally(e); + final Throwable failureCause = errorRef.get(); + completeResponseExceptionally(failureCause); if (!requestBodyCF.isDone()) { - requestBodyCF.completeExceptionally(errorRef.get()); // we may be sending the body.. + requestBodyCF.completeExceptionally(failureCause); // we may be sending the body.. } if (responseBodyCF != null) { - responseBodyCF.completeExceptionally(errorRef.get()); + responseBodyCF.completeExceptionally(failureCause); } } finally { connection.decrementStreamsCount(streamid); @@ -1698,7 +1717,35 @@ Throwable getCancelCause() { } final String dbgString() { - return connection.dbgString() + "/Stream("+streamid+")"; + final int id = streamid; + final String sid = id == 0 ? "?" : String.valueOf(id); + return connection.dbgString() + "/Stream(" + sid + ")"; + } + + /** + * An unprocessed exchange is one that hasn't been processed by a peer. The local end of the + * connection would be notified about such exchanges when it receives a GOAWAY frame with + * a stream id that tells which exchanges have been unprocessed. + * This method is called on such unprocessed exchanges and the implementation of this method + * will arrange for the request, corresponding to this exchange, to be retried afresh on a + * new connection. + */ + void closeAsUnprocessed() { + try { + // We arrange for the request to be retried on a new connection as allowed by the RFC-9113 + markUnprocessedByPeer(); + this.errorRef.compareAndSet(null, new IOException("request not processed by peer")); + if (debug.on()) { + debug.log("closing " + this.request + " as unprocessed by peer"); + } + // close the exchange and complete the response CF exceptionally + close(); + completeResponseExceptionally(this.errorRef.get()); + } finally { + // decrementStreamsCount isn't really needed but we do it to make sure + // the log messages, where these counts/states get reported, show the accurate state. + connection.decrementStreamsCount(streamid); + } } private class HeadersConsumer extends ValidatingHeadersConsumer implements DecodingCallback { diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java b/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java index 1df81640345..a30c21a7a7d 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2016, 2018, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2016, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -100,13 +100,16 @@ void removeStream(int streamid) { controllerLock.lock(); try { Integer old = streams.remove(streamid); - // Odd stream numbers (client streams) should have been registered. + // A client initiated stream might be closed (as unprocessed, due to a + // GOAWAY received on the connection) even before the stream is + // registered with this WindowController instance (when sending out request headers). + // Thus, for client initiated streams, we don't enforce the presence of the + // stream in the registered "streams" map. + // Even stream numbers (server streams - aka Push Streams) should // not be registered final boolean isClientStream = (streamid & 0x1) == 1; - if (old == null && isClientStream) { - throw new InternalError("Expected entry for streamid: " + streamid); - } else if (old != null && !isClientStream) { + if (old != null && !isClientStream) { throw new InternalError("Unexpected entry for streamid: " + streamid); } } finally { diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java b/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java index 45ec7553f5a..662e5dd98b3 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2018, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -57,7 +57,9 @@ int length() { @Override public String toString() { - return super.toString() + " Debugdata: " + new String(debugData, UTF_8); + return super.toString() + + " lastStreamId=" + lastStream + + ", Debugdata: " + new String(debugData, UTF_8); } public int getLastStream() { diff --git a/test/jdk/java/net/httpclient/http2/H2GoAwayTest.java b/test/jdk/java/net/httpclient/http2/H2GoAwayTest.java new file mode 100644 index 00000000000..755bb2e16cc --- /dev/null +++ b/test/jdk/java/net/httpclient/http2/H2GoAwayTest.java @@ -0,0 +1,336 @@ +/* + * Copyright (c) 2024, Oracle and/or its affiliates. All rights reserved. + * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. + * + * This code is free software; you can redistribute it and/or modify it + * under the terms of the GNU General Public License version 2 only, as + * published by the Free Software Foundation. + * + * This code is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License + * version 2 for more details (a copy is included in the LICENSE file that + * accompanied this code). + * + * You should have received a copy of the GNU General Public License version + * 2 along with this work; if not, write to the Free Software Foundation, + * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. + * + * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA + * or visit www.oracle.com if you need additional information or have any + * questions. + */ + +import java.io.IOException; +import java.io.OutputStream; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.net.http.HttpResponse.BodyHandlers; +import java.util.ArrayList; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.net.ssl.SSLContext; + +import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestExchange; +import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestHandler; +import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestServer; +import jdk.test.lib.net.SimpleSSLContext; +import jdk.test.lib.net.URIBuilder; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import static java.net.http.HttpClient.Version.HTTP_2; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.fail; + +/* + * @test + * @bug 8335181 + * @summary verify that the HttpClient correctly handles incoming GOAWAY frames and + * retries any unprocessed requests on a new connection + * @library /test/lib /test/jdk/java/net/httpclient/lib + * @build jdk.httpclient.test.lib.common.HttpServerAdapters + * jdk.test.lib.net.SimpleSSLContext + * @run junit H2GoAwayTest + */ +public class H2GoAwayTest { + private static final String REQ_PATH = "/test"; + private static HttpTestServer server; + private static String REQ_URI_BASE; + private static SSLContext sslCtx; + + @BeforeAll + static void beforeAll() throws Exception { + sslCtx = new SimpleSSLContext().get(); + assertNotNull(sslCtx, "SSLContext couldn't be created"); + server = HttpTestServer.create(HTTP_2, sslCtx); + server.addHandler(new Handler(), REQ_PATH); + server.start(); + System.out.println("Server started at " + server.getAddress()); + REQ_URI_BASE = URIBuilder.newBuilder().scheme("https") + .loopback() + .port(server.getAddress().getPort()) + .path(REQ_PATH) + .build().toString(); + } + + @AfterAll + static void afterAll() { + if (server != null) { + System.out.println("Stopping server at " + server.getAddress()); + server.stop(); + } + } + + /** + * Verifies that when several requests are sent using send() and the server + * connection is configured to send a GOAWAY after processing only a few requests, then + * the remaining requests are retried on a different connection + */ + @Test + public void testSequential() throws Exception { + final LimitedPerConnRequestApprover reqApprover = new LimitedPerConnRequestApprover(); + server.setRequestApprover(reqApprover::allowNewRequest); + try (final HttpClient client = HttpClient.newBuilder().version(HTTP_2) + .sslContext(sslCtx).build()) { + final String[] reqMethods = {"HEAD", "GET", "POST"}; + for (final String reqMethod : reqMethods) { + final int numReqs = LimitedPerConnRequestApprover.MAX_REQS_PER_CONN + 3; + final Set connectionKeys = new LinkedHashSet<>(); + for (int i = 1; i <= numReqs; i++) { + final URI reqURI = new URI(REQ_URI_BASE + "?seq&" + reqMethod + "=" + i); + final HttpRequest req = HttpRequest.newBuilder() + .uri(reqURI) + .method(reqMethod, HttpRequest.BodyPublishers.noBody()) + .build(); + System.out.println("initiating request " + req); + final HttpResponse resp = client.send(req, BodyHandlers.ofString()); + final String respBody = resp.body(); + System.out.println("received response: " + respBody); + assertEquals(200, resp.statusCode(), + "unexpected status code for request " + resp.request()); + // response body is the logical key of the connection on which the + // request was handled + connectionKeys.add(respBody); + } + System.out.println("connections involved in handling the requests: " + + connectionKeys); + // all requests have finished, we now just do a basic check that + // more than one connection was involved in processing these requests + assertEquals(2, connectionKeys.size(), + "unexpected number of connections " + connectionKeys); + } + } finally { + server.setRequestApprover(null); // reset + } + } + + /** + * Verifies that when a server responds with a GOAWAY and then never processes the new retried + * requests on a new connection too, then the application code receives the request failure. + * This tests the send() API of the HttpClient. + */ + @Test + public void testUnprocessedRaisesException() throws Exception { + try (final HttpClient client = HttpClient.newBuilder().version(HTTP_2) + .sslContext(sslCtx).build()) { + final Random random = new Random(); + final String[] reqMethods = {"HEAD", "GET", "POST"}; + for (final String reqMethod : reqMethods) { + final int maxAllowedReqs = 2; + final int numReqs = maxAllowedReqs + 3; // 3 more requests than max allowed + // configure the approver + final LimitedRequestApprover reqApprover = new LimitedRequestApprover(maxAllowedReqs); + server.setRequestApprover(reqApprover::allowNewRequest); + try { + int numSuccess = 0; + int numFailed = 0; + for (int i = 1; i <= numReqs; i++) { + final String reqQueryPart = "?sync&" + reqMethod + "=" + i; + final URI reqURI = new URI(REQ_URI_BASE + reqQueryPart); + final HttpRequest req = HttpRequest.newBuilder() + .uri(reqURI) + .method(reqMethod, HttpRequest.BodyPublishers.noBody()) + .build(); + System.out.println("initiating request " + req); + if (i <= maxAllowedReqs) { + // expected to successfully complete + numSuccess++; + final HttpResponse resp = client.send(req, BodyHandlers.ofString()); + final String respBody = resp.body(); + System.out.println("received response: " + respBody); + assertEquals(200, resp.statusCode(), + "unexpected status code for request " + resp.request()); + } else { + // expected to fail as unprocessed + try { + final HttpResponse resp = client.send(req, BodyHandlers.ofString()); + fail("Request was expected to fail as unprocessed," + + " but got response: " + resp.body() + ", status code: " + + resp.statusCode()); + } catch (IOException ioe) { + // verify it failed for the right reason + if (ioe.getMessage() == null + || !ioe.getMessage().contains("request not processed by peer")) { + // propagate the original failure + throw ioe; + } + numFailed++; // failed due to right reason + System.out.println("received expected failure: " + ioe + + ", for request " + reqURI); + } + } + } + // verify the correct number of requests succeeded/failed + assertEquals(maxAllowedReqs, numSuccess, "unexpected number of requests succeeded"); + assertEquals((numReqs - maxAllowedReqs), numFailed, "unexpected number of requests failed"); + } finally { + server.setRequestApprover(null); // reset + } + } + } + } + + /** + * Verifies that when a server responds with a GOAWAY and then never processes the new retried + * requests on a new connection too, then the application code receives the request failure. + * This tests the sendAsync() API of the HttpClient. + */ + @Test + public void testUnprocessedRaisesExceptionAsync() throws Throwable { + try (final HttpClient client = HttpClient.newBuilder().version(HTTP_2) + .sslContext(sslCtx).build()) { + final Random random = new Random(); + final String[] reqMethods = {"HEAD", "GET", "POST"}; + for (final String reqMethod : reqMethods) { + final int maxAllowedReqs = 2; + final int numReqs = maxAllowedReqs + 3; // 3 more requests than max allowed + // configure the approver + final LimitedRequestApprover reqApprover = new LimitedRequestApprover(maxAllowedReqs); + server.setRequestApprover(reqApprover::allowNewRequest); + try { + final List>> futures = new ArrayList<>(); + for (int i = 1; i <= numReqs; i++) { + final URI reqURI = new URI(REQ_URI_BASE + "?async&" + reqMethod + "=" + i); + final HttpRequest req = HttpRequest.newBuilder() + .uri(reqURI) + .method(reqMethod, HttpRequest.BodyPublishers.noBody()) + .build(); + System.out.println("initiating request " + req); + final Future> f = client.sendAsync(req, BodyHandlers.ofString()); + futures.add(f); + } + // wait for responses + int numFailed = 0; + int numSuccess = 0; + for (int i = 1; i <= numReqs; i++) { + final String reqQueryPart = "?async&" + reqMethod + "=" + i; + try { + System.out.println("waiting response of request " + + REQ_URI_BASE + reqQueryPart); + final HttpResponse resp = futures.get(i - 1).get(); + numSuccess++; + final String respBody = resp.body(); + System.out.println("request: " + resp.request() + + ", received response: " + respBody); + assertEquals(200, resp.statusCode(), + "unexpected status code for request " + resp.request()); + } catch (ExecutionException ee) { + final Throwable cause = ee.getCause(); + if (!(cause instanceof IOException ioe)) { + throw cause; + } + // verify it failed for the right reason + if (ioe.getMessage() == null + || !ioe.getMessage().contains("request not processed by peer")) { + // propagate the original failure + throw ioe; + } + numFailed++; // failed due to the right reason + System.out.println("received expected failure: " + ioe + + ", for request " + REQ_URI_BASE + reqQueryPart); + } + } + // verify the correct number of requests succeeded/failed + assertEquals(maxAllowedReqs, numSuccess, "unexpected number of requests succeeded"); + assertEquals((numReqs - maxAllowedReqs), numFailed, "unexpected number of requests failed"); + } finally { + server.setRequestApprover(null); // reset + } + } + } + } + + // only allows fixed number of requests, irrespective of which server connection handles + // it. requests that are rejected will either be sent a GOAWAY on the connection + // or a RST_FRAME with a REFUSED_STREAM on the stream + private static final class LimitedRequestApprover { + private final int maxAllowedReqs; + private final AtomicInteger numApproved = new AtomicInteger(); + + private LimitedRequestApprover(final int maxAllowedReqs) { + this.maxAllowedReqs = maxAllowedReqs; + } + + public boolean allowNewRequest(final String serverConnKey) { + final int approved = numApproved.incrementAndGet(); + return approved <= maxAllowedReqs; + } + } + + // allows a certain number of requests per server connection. + // requests that are rejected will either be sent a GOAWAY on the connection + // or a RST_FRAME with a REFUSED_STREAM on the stream + private static final class LimitedPerConnRequestApprover { + private static final int MAX_REQS_PER_CONN = 6; + private final Map numApproved = + new ConcurrentHashMap<>(); + private final Map numDisapproved = + new ConcurrentHashMap<>(); + + public boolean allowNewRequest(final String serverConnKey) { + final AtomicInteger approved = numApproved.computeIfAbsent(serverConnKey, + (k) -> new AtomicInteger()); + int curr = approved.get(); + while (curr < MAX_REQS_PER_CONN) { + if (approved.compareAndSet(curr, curr + 1)) { + return true; // new request allowed + } + curr = approved.get(); + } + final AtomicInteger disapproved = numDisapproved.computeIfAbsent(serverConnKey, + (k) -> new AtomicInteger()); + final int numUnprocessed = disapproved.incrementAndGet(); + System.out.println(approved.get() + " processed, " + + numUnprocessed + " unprocessed requests on connection " + serverConnKey); + return false; + } + } + + private static final class Handler implements HttpTestHandler { + + @Override + public void handle(final HttpTestExchange exchange) throws IOException { + final String connectionKey = exchange.getConnectionKey(); + System.out.println("responding to request: " + exchange.getRequestURI() + + " on connection " + connectionKey); + final byte[] response = connectionKey.getBytes(UTF_8); + exchange.sendResponseHeaders(200, response.length); + try (final OutputStream os = exchange.getResponseBody()) { + os.write(response); + } + } + } +} diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java index 98da7d5533f..b91952ba410 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java @@ -58,6 +58,7 @@ import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; import java.util.logging.Level; import java.util.logging.Logger; import java.util.stream.Collectors; @@ -240,6 +241,7 @@ public static abstract class HttpTestExchange implements AutoCloseable { public abstract void close(); public abstract InetSocketAddress getRemoteAddress(); public abstract InetSocketAddress getLocalAddress(); + public abstract String getConnectionKey(); public void serverPush(URI uri, HttpHeaders headers, byte[] body) { ByteArrayInputStream bais = new ByteArrayInputStream(body); serverPush(uri, headers, bais); @@ -254,7 +256,7 @@ public static HttpTestExchange of(HttpExchange exchange) { return new Http1TestExchange(exchange); } public static HttpTestExchange of(Http2TestExchange exchange) { - return new Http2TestExchangeImpl(exchange); + return new H2ExchangeImpl(exchange); } abstract void doFilter(Filter.Chain chain) throws IOException; @@ -310,15 +312,21 @@ public InetSocketAddress getLocalAddress() { public URI getRequestURI() { return exchange.getRequestURI(); } @Override public String getRequestMethod() { return exchange.getRequestMethod(); } + + @Override + public String getConnectionKey() { + return exchange.getLocalAddress() + "->" + exchange.getRemoteAddress(); + } + @Override public String toString() { return this.getClass().getSimpleName() + ": " + exchange.toString(); } } - private static final class Http2TestExchangeImpl extends HttpTestExchange { + private static final class H2ExchangeImpl extends HttpTestExchange { private final Http2TestExchange exchange; - Http2TestExchangeImpl(Http2TestExchange exch) { + H2ExchangeImpl(Http2TestExchange exch) { this.exchange = exch; } @Override @@ -371,6 +379,11 @@ public InetSocketAddress getLocalAddress() { return exchange.getLocalAddress(); } + @Override + public String getConnectionKey() { + return exchange.getConnectionKey(); + } + @Override public URI getRequestURI() { return exchange.getRequestURI(); } @Override @@ -716,6 +729,7 @@ static void enableLogging() { public abstract HttpTestContext addHandler(HttpTestHandler handler, String root); public abstract InetSocketAddress getAddress(); public abstract Version getVersion(); + public abstract void setRequestApprover(final Predicate approver); public String serverAuthority() { InetSocketAddress address = getAddress(); @@ -864,6 +878,11 @@ public InetSocketAddress getAddress() { impl.getAddress().getPort()); } public Version getVersion() { return Version.HTTP_1_1; } + + @Override + public void setRequestApprover(final Predicate approver) { + throw new UnsupportedOperationException("not supported"); + } } private static class Http1TestContext extends HttpTestContext { @@ -915,6 +934,11 @@ public InetSocketAddress getAddress() { impl.getAddress().getPort()); } public Version getVersion() { return Version.HTTP_2; } + + @Override + public void setRequestApprover(final Predicate approver) { + this.impl.setRequestApprover(approver); + } } private static class Http2TestContext diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java index 1a8b5d92af5..828c939f53f 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2017, 2023, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2017, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -84,4 +84,10 @@ default void sendFrames(List frames) throws IOException { * It may also complete exceptionally */ CompletableFuture sendPing(); + + /** + * {@return the identification of the connection on which this exchange is being + * processed} + */ + String getConnectionKey(); } diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java index 0abcd51af63..c867b8bf040 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java @@ -227,6 +227,11 @@ public void serverPush(URI uri, HttpHeaders headers, InputStream content) { } } + @Override + public String getConnectionKey() { + return conn.connectionKey(); + } + private boolean isHeadRequest() { return HEAD.equalsIgnoreCase(getRequestMethod()); } diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java index ffa05d5c11c..ca7e7d3dc30 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2023, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -32,6 +32,8 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; + import javax.net.ServerSocketFactory; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLParameters; @@ -59,6 +61,8 @@ public class Http2TestServer implements AutoCloseable { final Set connections; final Properties properties; final String name; + // request approver which takes the server connection key as the input + private volatile Predicate newRequestApprover; private static ThreadFactory defaultThreadFac = (Runnable r) -> { @@ -285,6 +289,14 @@ public String serverName() { return serverName; } + public void setRequestApprover(final Predicate approver) { + this.newRequestApprover = approver; + } + + Predicate getRequestApprover() { + return this.newRequestApprover; + } + private synchronized void putConnection(InetSocketAddress addr, Http2TestServerConnection c) { if (!stopping) connections.add(c); diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java index b087a037222..40cc1332c76 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java @@ -77,15 +77,20 @@ import java.util.Optional; import java.util.Properties; import java.util.Random; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiPredicate; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import java.util.function.Consumer; +import java.util.function.Predicate; import static java.nio.charset.StandardCharsets.ISO_8859_1; import static java.nio.charset.StandardCharsets.UTF_8; +import static jdk.internal.net.http.frame.ErrorFrame.REFUSED_STREAM; import static jdk.internal.net.http.frame.SettingsFrame.DEFAULT_MAX_FRAME_SIZE; import static jdk.internal.net.http.frame.SettingsFrame.HEADER_TABLE_SIZE; @@ -115,6 +120,10 @@ public class Http2TestServerConnection { volatile boolean stopping; volatile int nextPushStreamId = 2; ConcurrentLinkedQueue pings = new ConcurrentLinkedQueue<>(); + // the max stream id of a processed H2 request. -1 implies none were processed. + private final AtomicInteger maxProcessedRequestStreamId = new AtomicInteger(-1); + // the stream id that was sent in a GOAWAY frame. -1 implies no GOAWAY frame was sent. + private final AtomicInteger goAwayRequestStreamId = new AtomicInteger(-1); final static ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0); final static byte[] EMPTY_BARRAY = new byte[0]; @@ -239,11 +248,29 @@ CompletableFuture sendPing() { return ping.response(); } - void goAway(int error) throws IOException { - int laststream = nextstream >= 3 ? nextstream - 2 : 1; - - GoAwayFrame go = new GoAwayFrame(laststream, error); - outputQ.put(go); + private void sendGoAway(final int error) throws IOException { + int maxProcessedStreamId = maxProcessedRequestStreamId.get(); + if (maxProcessedStreamId == -1) { + maxProcessedStreamId = 0; + } + boolean send = false; + int currentGoAwayReqStrmId = goAwayRequestStreamId.get(); + // update the last processed stream id and send a goaway frame if the new last processed + // stream id is lesser than the last processed stream id sent in + // a previous goaway frame (if any) + while (currentGoAwayReqStrmId == -1 || maxProcessedStreamId < currentGoAwayReqStrmId) { + if (goAwayRequestStreamId.compareAndSet(currentGoAwayReqStrmId, maxProcessedStreamId)) { + send = true; + break; + } + currentGoAwayReqStrmId = goAwayRequestStreamId.get(); + } + if (!send) { + return; + } + final GoAwayFrame frame = new GoAwayFrame(maxProcessedStreamId, error); + outputQ.put(frame); + System.err.println("Sending GOAWAY frame " + frame + " from server connection " + this); } /** @@ -336,8 +363,9 @@ void close(int error) { q.orderlyClose(); }); try { - if (error != -1) - goAway(error); + if (error != -1) { + sendGoAway(error); + } outputQ.orderlyClose(); socket.close(); } catch (Exception e) { @@ -619,6 +647,14 @@ void createPrimordialStream(Http1InitialRequest request) throws IOException { path = path + "?" + uri.getRawQuery(); headersBuilder.setHeader(":path", path); + // skip processing the request if configured to do so + final String connKey = connectionKey(); + if (!shouldProcessNewHTTPRequest(connKey)) { + System.err.println("Rejecting primordial stream 1 and sending GOAWAY" + + " on server connection " + connKey + ", for request: " + path); + sendGoAway(ErrorFrame.NO_ERROR); + return; + } Queue q = new Queue(sentinel); byte[] body = getRequestBody(request); addHeaders(getHeaders(request.headers), headersBuilder); @@ -627,11 +663,24 @@ void createPrimordialStream(Http1InitialRequest request) throws IOException { addRequestBodyToQueue(body, q); streams.put(1, q); + maxProcessedRequestStreamId.set(1); exec.submit(() -> { handleRequest(headers, q, 1, true /*complete request has been read*/); }); } + private boolean shouldProcessNewHTTPRequest(final String serverConnKey) { + final Predicate approver = this.server.getRequestApprover(); + if (approver == null) { + return true; // process the request + } + return approver.test(serverConnKey); + } + + final String connectionKey() { + return this.server.getAddress() + "->" + this.socket.getRemoteSocketAddress(); + } + // all other streams created here @SuppressWarnings({"rawtypes","unchecked"}) void createStream(HeaderFrame frame) throws IOException { @@ -639,7 +688,7 @@ void createStream(HeaderFrame frame) throws IOException { frames.add(frame); int streamid = frame.streamid(); if (streamid != nextstream) { - throw new IOException("unexpected stream id"); + throw new IOException("unexpected stream id: " + streamid); } nextstream += 2; @@ -670,12 +719,30 @@ void createStream(HeaderFrame frame) throws IOException { throw new IOException("Unexpected Upgrade in headers:" + headers); } disallowedHeader = headers.firstValue("HTTP2-Settings"); - if (disallowedHeader.isPresent()) + if (disallowedHeader.isPresent()) { throw new IOException("Unexpected HTTP2-Settings in headers:" + headers); + } - + // skip processing the request if the server is configured to do so + final String connKey = connectionKey(); + final String path = headers.firstValue(":path").orElse(""); + if (!shouldProcessNewHTTPRequest(connKey)) { + System.err.println("Rejecting stream " + streamid + + " and sending GOAWAY on server connection " + + connKey + ", for request: " + path); + sendGoAway(ErrorFrame.NO_ERROR); + return; + } Queue q = new Queue(sentinel); streams.put(streamid, q); + // keep track of the largest request id that we have processed + int currentLargest = maxProcessedRequestStreamId.get(); + while (streamid > currentLargest) { + if (maxProcessedRequestStreamId.compareAndSet(currentLargest, streamid)) { + break; + } + currentLargest = maxProcessedRequestStreamId.get(); + } exec.submit(() -> { handleRequest(headers, q, streamid, endStreamReceived); }); @@ -778,6 +845,8 @@ void readLoop() { while (!stopping) { Http2Frame frame = readFrameImpl(); if (frame == null) { + System.err.println("EOF reached on connection " + connectionKey() + + ", will no longer accept incoming frames"); closeIncoming(); return; } @@ -801,6 +870,17 @@ void readLoop() { // TODO: close connection continue; } else { + final int streamId = frame.streamid(); + final int finalProcessedStreamId = goAwayRequestStreamId.get(); + // if we already sent a goaway, then don't create new streams with + // higher stream ids. + if (finalProcessedStreamId != -1 && streamId > finalProcessedStreamId) { + System.err.println(connectionKey() + " resetting stream " + streamId + + " as REFUSED_STREAM"); + final ResetFrame rst = new ResetFrame(streamId, REFUSED_STREAM); + outputQ.put(rst); + continue; + } createStream((HeadersFrame) frame); } } else { From 480336754274ea62ab2e9b076dcec91f7d483c25 Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Thu, 20 Feb 2025 15:42:31 -0800 Subject: [PATCH 3/9] Remove extra import --- .../jdk/httpclient/test/lib/http2/Http2TestServerConnection.java | 1 - 1 file changed, 1 deletion(-) diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java index 40cc1332c76..056fb669817 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java @@ -82,7 +82,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BiPredicate; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import java.util.function.Consumer; From c0f83e97475d2b73cbc1d03e5fb02d2eb64565d3 Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Thu, 20 Feb 2025 16:06:50 -0800 Subject: [PATCH 4/9] Backport 6273ab97dc1a0d3c1f51ba94694d9594dd7593d4 --- .../jdk/internal/net/http/HttpClientImpl.java | 3 ++- .../share/classes/jdk/internal/net/http/Stream.java | 9 +++++---- .../net/httpclient/AbstractThrowingSubscribers.java | 12 ++++++++++-- test/jdk/java/net/httpclient/ReferenceTracker.java | 8 ++++++++ 4 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java b/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java index d88923192e5..9a8b901df7d 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java @@ -577,8 +577,9 @@ public boolean registerSubscriber(HttpBodySubscriberWrapper subscriber) { if (debug.on()) { debug.log("body subscriber registered: " + count); } + return true; } - return true; + return false; } } finally { selmgr.unlock(); diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java b/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java index 5ae9f707d7c..30015484c7b 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java @@ -191,7 +191,7 @@ private void schedule() { if (debug.on()) debug.log("subscribing user subscriber"); subscriber.onSubscribe(userSubscription); } - while (!inputQ.isEmpty()) { + while (!inputQ.isEmpty() && errorRef.get() == null) { Http2Frame frame = inputQ.peek(); if (frame instanceof ResetFrame rf) { inputQ.remove(); @@ -425,6 +425,10 @@ private void sendDataFrame(DataFrame frame) { // pushes entire response body into response subscriber // blocking when required by local or remote flow control CompletableFuture receiveData(BodySubscriber bodySubscriber, Executor executor) { + // ensure that the body subscriber will be subscribed and onError() is + // invoked + pendingResponseSubscriber = bodySubscriber; + // We want to allow the subscriber's getBody() method to block so it // can work with InputStreams. So, we offload execution. responseBodyCF = ResponseSubscribers.getBodyAsync(executor, bodySubscriber, @@ -435,9 +439,6 @@ CompletableFuture receiveData(BodySubscriber bodySubscriber, Executor exec responseBodyCF.completeExceptionally(t); } - // ensure that the body subscriber will be subscribed and onError() is - // invoked - pendingResponseSubscriber = bodySubscriber; sched.runOrSchedule(); // in case data waiting already to be processed, or error return responseBodyCF; diff --git a/test/jdk/java/net/httpclient/AbstractThrowingSubscribers.java b/test/jdk/java/net/httpclient/AbstractThrowingSubscribers.java index 077ae346462..dc7ca3fe9d5 100644 --- a/test/jdk/java/net/httpclient/AbstractThrowingSubscribers.java +++ b/test/jdk/java/net/httpclient/AbstractThrowingSubscribers.java @@ -474,6 +474,7 @@ private void testThrowing(String uri, boolean sameClient, if (response != null) { finisher.finish(where, response, thrower); } + var tracker = TRACKER.getTracker(client); if (!sameClient) { // Wait for the client to be garbage collected. // we use the ReferenceTracker API rather than HttpClient::close here, @@ -482,7 +483,6 @@ private void testThrowing(String uri, boolean sameClient, // By using the ReferenceTracker, we will get some diagnosis about what // is keeping the client alive if it doesn't get GC'ed within the // expected time frame. - var tracker = TRACKER.getTracker(client); client = null; System.gc(); System.out.println(now() + "waiting for client to shutdown: " + tracker.getName()); @@ -491,6 +491,14 @@ private void testThrowing(String uri, boolean sameClient, if (error != null) throw error; System.out.println(now() + "client shutdown normally: " + tracker.getName()); System.err.println(now() + "client shutdown normally: " + tracker.getName()); + } else { + System.out.println(now() + "waiting for operation to finish: " + tracker.getName()); + System.err.println(now() + "waiting for operation to finish: " + tracker.getName()); + var error = TRACKER.checkFinished(tracker, 10000); + if (error != null) throw error; + System.out.println(now() + "operation finished normally: " + tracker.getName()); + System.err.println(now() + "operation finished normally: " + tracker.getName()); + } } } @@ -800,7 +808,7 @@ public void teardown() throws Exception { sharedClient == null ? null : sharedClient.toString(); sharedClient = null; Thread.sleep(100); - AssertionError fail = TRACKER.check(500); + AssertionError fail = TRACKER.check(5000); try { httpTestServer.stop(); httpsTestServer.stop(); diff --git a/test/jdk/java/net/httpclient/ReferenceTracker.java b/test/jdk/java/net/httpclient/ReferenceTracker.java index dbc4a128675..d7e16d01201 100644 --- a/test/jdk/java/net/httpclient/ReferenceTracker.java +++ b/test/jdk/java/net/httpclient/ReferenceTracker.java @@ -115,6 +115,14 @@ public AssertionError check(Tracker tracker, long graceDelayMs) { "outstanding operations or unreleased resources", true); } + public AssertionError checkFinished(Tracker tracker, long graceDelayMs) { + Predicate hasOperations = (t) -> t.getOutstandingOperations() > 0; + Predicate hasSubscribers = (t) -> t.getOutstandingSubscribers() > 0; + return check(tracker, graceDelayMs, + hasOperations.or(hasSubscribers), + "outstanding operations or unreleased resources", false); + } + public AssertionError check(long graceDelayMs) { Predicate hasOperations = (t) -> t.getOutstandingOperations() > 0; Predicate hasSubscribers = (t) -> t.getOutstandingSubscribers() > 0; From 0321dff4cd0c72c01508a3aa3308094686c1c495 Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Fri, 21 Feb 2025 13:07:08 -0800 Subject: [PATCH 5/9] Revert "Remove extra import" This reverts commit 480336754274ea62ab2e9b076dcec91f7d483c25. --- .../jdk/httpclient/test/lib/http2/Http2TestServerConnection.java | 1 + 1 file changed, 1 insertion(+) diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java index 056fb669817..40cc1332c76 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java @@ -82,6 +82,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiPredicate; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import java.util.function.Consumer; From 8a6b57bfe5aaf58f566bb8b2d0222fceabf53d04 Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Fri, 21 Feb 2025 13:08:20 -0800 Subject: [PATCH 6/9] Revert "Backport 720b44648bcff997278af92746f942b2425298a5" This reverts commit a5b7bb4d432c868d598c35f54306e40747071ad7. --- .../jdk/internal/net/http/ExchangeImpl.java | 15 +- .../internal/net/http/Http2Connection.java | 55 +-- .../jdk/internal/net/http/MultiExchange.java | 28 +- .../classes/jdk/internal/net/http/Stream.java | 67 +--- .../internal/net/http/WindowController.java | 13 +- .../internal/net/http/frame/GoAwayFrame.java | 6 +- .../net/httpclient/http2/H2GoAwayTest.java | 336 ------------------ .../test/lib/common/HttpServerAdapters.java | 30 +- .../test/lib/http2/Http2TestExchange.java | 8 +- .../test/lib/http2/Http2TestExchangeImpl.java | 5 - .../test/lib/http2/Http2TestServer.java | 14 +- .../lib/http2/Http2TestServerConnection.java | 100 +----- 12 files changed, 54 insertions(+), 623 deletions(-) delete mode 100644 test/jdk/java/net/httpclient/http2/H2GoAwayTest.java diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java b/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java index 71e669b7ce1..852742b6606 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/ExchangeImpl.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2022, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -58,10 +58,6 @@ abstract class ExchangeImpl { final Exchange exchange; - // this will be set to true only when the peer explicitly states (through a GOAWAY frame or - // a relevant error code in reset frame) that the corresponding stream (id) wasn't processed - private volatile boolean unprocessedByPeer; - ExchangeImpl(Exchange e) { // e == null means a http/2 pushed stream this.exchange = e; @@ -268,13 +264,4 @@ void upgraded() { } // Called when server returns non 100 response to // an Expect-Continue void expectContinueFailed(int rcode) { } - - final boolean isUnprocessedByPeer() { - return this.unprocessedByPeer; - } - - // Marks the exchange as unprocessed by the peer - final void markUnprocessedByPeer() { - this.unprocessedByPeer = true; - } } diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java b/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java index bb7a17a89cb..84c09d6bca6 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java @@ -49,7 +49,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Flow; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -397,7 +396,6 @@ private record PushContinuationState(PushPromiseDecoder pushContDecoder, PushPro private final String key; // for HttpClientImpl.connections map private final FramesDecoder framesDecoder; private final FramesEncoder framesEncoder = new FramesEncoder(); - private final AtomicLong lastProcessedStreamInGoAway = new AtomicLong(-1); /** * Send Window controller for both connection and stream windows. @@ -804,9 +802,7 @@ final int maxConcurrentServerInitiatedStreams() { void close() { if (markHalfClosedLocal()) { - // we send a GOAWAY frame only if the remote side hasn't already indicated - // the intention to close the connection by previously sending a GOAWAY of its own - if (connection.channel().isOpen() && !isMarked(closedState, HALF_CLOSED_REMOTE)) { + if (connection.channel().isOpen()) { Log.logTrace("Closing HTTP/2 connection: to {0}", connection.address()); GoAwayFrame f = new GoAwayFrame(0, ErrorFrame.NO_ERROR, @@ -1358,46 +1354,13 @@ private void handlePing(PingFrame frame) sendUnorderedFrame(frame); } - private void handleGoAway(final GoAwayFrame frame) { - final long lastProcessedStream = frame.getLastStream(); - assert lastProcessedStream >= 0 : "unexpected last stream id: " - + lastProcessedStream + " in GOAWAY frame"; - - markHalfClosedRemote(); - setFinalStream(); // don't allow any new streams on this connection - if (debug.on()) { - debug.log("processing incoming GOAWAY with last processed stream id:%s in frame %s", - lastProcessedStream, frame); - } - // see if this connection has previously received a GOAWAY from the peer and if yes - // then check if this new last processed stream id is lesser than the previous - // known last processed stream id. Only update the last processed stream id if the new - // one is lesser than the previous one. - long prevLastProcessed = lastProcessedStreamInGoAway.get(); - while (prevLastProcessed == -1 || lastProcessedStream < prevLastProcessed) { - if (lastProcessedStreamInGoAway.compareAndSet(prevLastProcessed, - lastProcessedStream)) { - break; - } - prevLastProcessed = lastProcessedStreamInGoAway.get(); - } - handlePeerUnprocessedStreams(lastProcessedStreamInGoAway.get()); - } - - private void handlePeerUnprocessedStreams(final long lastProcessedStream) { - final AtomicInteger numClosed = new AtomicInteger(); // atomic merely to allow usage within lambda - streams.forEach((id, exchange) -> { - if (id > lastProcessedStream) { - // any streams with an stream id higher than the last processed stream - // can be retried (on a new connection). we close the exchange as unprocessed - // to facilitate the retrying. - client2.client().theExecutor().ensureExecutedAsync(exchange::closeAsUnprocessed); - numClosed.incrementAndGet(); - } - }); - if (debug.on()) { - debug.log(numClosed.get() + " stream(s), with id greater than " + lastProcessedStream - + ", will be closed as unprocessed"); + private void handleGoAway(GoAwayFrame frame) + throws IOException + { + if (markHalfClosedLRemote()) { + shutdown(new IOException( + connection.channel().getLocalAddress() + + ": GOAWAY received")); } } @@ -1948,7 +1911,7 @@ private boolean markHalfClosedLocal() { return markClosedState(HALF_CLOSED_LOCAL); } - private boolean markHalfClosedRemote() { + private boolean markHalfClosedLRemote() { return markClosedState(HALF_CLOSED_REMOTE); } diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java index dbbcb7da19c..b3bc83e93de 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2023, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -90,7 +90,7 @@ class MultiExchange implements Cancelable { Exchange exchange; // the current exchange Exchange previous; volatile Throwable retryCause; - volatile boolean retriedOnce; + volatile boolean expiredOnce; volatile HttpResponse response; // Maximum number of times a request will be retried/redirected @@ -459,7 +459,7 @@ private CompletableFuture responseAsyncImpl() { return exch.ignoreBody().handle((r,t) -> { previousreq = currentreq; currentreq = newrequest; - retriedOnce = false; + expiredOnce = false; setExchange(new Exchange<>(currentreq, this, acc)); return responseAsyncImpl(); }).thenCompose(Function.identity()); @@ -472,7 +472,7 @@ private CompletableFuture responseAsyncImpl() { return completedFuture(response); } // all exceptions thrown are handled here - CompletableFuture errorCF = getExceptionalCF(ex, exch.exchImpl); + CompletableFuture errorCF = getExceptionalCF(ex); if (errorCF == null) { return responseAsyncImpl(); } else { @@ -544,38 +544,34 @@ private Throwable retryCause(Throwable t) { * Takes a Throwable and returns a suitable CompletableFuture that is * completed exceptionally, or null. */ - private CompletableFuture getExceptionalCF(Throwable t, ExchangeImpl exchImpl) { + private CompletableFuture getExceptionalCF(Throwable t) { if ((t instanceof CompletionException) || (t instanceof ExecutionException)) { if (t.getCause() != null) { t = t.getCause(); } } - final boolean retryAsUnprocessed = exchImpl != null && exchImpl.isUnprocessedByPeer(); if (cancelled && !requestCancelled() && t instanceof IOException) { if (!(t instanceof HttpTimeoutException)) { t = toTimeoutException((IOException)t); } - } else if (retryAsUnprocessed || retryOnFailure(t)) { + } else if (retryOnFailure(t)) { Throwable cause = retryCause(t); if (!(t instanceof ConnectException)) { // we may need to start a new connection, and if so // we want to start with a fresh connect timeout again. if (connectTimeout != null) connectTimeout.reset(); - if (!retryAsUnprocessed && !canRetryRequest(currentreq)) { - // a (peer) processed request which cannot be retried, fail with - // the original cause - return failedFuture(cause); + if (!canRetryRequest(currentreq)) { + return failedFuture(cause); // fails with original cause } } // ConnectException: retry, but don't reset the connectTimeout. // allow the retry mechanism to do its work retryCause = cause; - if (!retriedOnce) { + if (!expiredOnce) { if (debug.on()) - debug.log(t.getClass().getSimpleName() - + " (async): retrying " + currentreq + " due to: ", t); - retriedOnce = true; + debug.log(t.getClass().getSimpleName() + " (async): retrying...", t); + expiredOnce = true; // The connection was abruptly closed. // We return null to retry the same request a second time. // The request filters have already been applied to the @@ -586,7 +582,7 @@ private CompletableFuture getExceptionalCF(Throwable t, ExchangeImpl subscriber) { stateLock.unlock(); } try { - final int error = frame.getErrorCode(); - // A REFUSED_STREAM error code implies that the stream wasn't processed by the - // peer and the client is free to retry the request afresh. - if (error == ErrorFrame.REFUSED_STREAM) { - // Here we arrange for the request to be retried. Note that we don't call - // closeAsUnprocessed() method here because the "closed" state is already set - // to true a few lines above and calling close() from within - // closeAsUnprocessed() will end up being a no-op. We instead do the additional - // bookkeeping here. - markUnprocessedByPeer(); - errorRef.compareAndSet(null, new IOException("request not processed by peer")); - if (debug.on()) { - debug.log("request unprocessed by peer (REFUSED_STREAM) " + this.request); - } - } else { - final String reason = ErrorFrame.stringForCode(error); - final IOException failureCause = new IOException("Received RST_STREAM: " + reason); - if (debug.on()) { - debug.log(streamid + " received RST_STREAM with code: " + reason); - } - if (errorRef.compareAndSet(null, failureCause)) { - if (subscriber != null) { - subscriber.onError(failureCause); - } + int error = frame.getErrorCode(); + IOException e = new IOException("Received RST_STREAM: " + + ErrorFrame.stringForCode(error)); + if (errorRef.compareAndSet(null, e)) { + if (subscriber != null) { + subscriber.onError(e); } } - final Throwable failureCause = errorRef.get(); - completeResponseExceptionally(failureCause); + completeResponseExceptionally(e); if (!requestBodyCF.isDone()) { - requestBodyCF.completeExceptionally(failureCause); // we may be sending the body.. + requestBodyCF.completeExceptionally(errorRef.get()); // we may be sending the body.. } if (responseBodyCF != null) { - responseBodyCF.completeExceptionally(failureCause); + responseBodyCF.completeExceptionally(errorRef.get()); } } finally { connection.decrementStreamsCount(streamid); @@ -1718,35 +1699,7 @@ Throwable getCancelCause() { } final String dbgString() { - final int id = streamid; - final String sid = id == 0 ? "?" : String.valueOf(id); - return connection.dbgString() + "/Stream(" + sid + ")"; - } - - /** - * An unprocessed exchange is one that hasn't been processed by a peer. The local end of the - * connection would be notified about such exchanges when it receives a GOAWAY frame with - * a stream id that tells which exchanges have been unprocessed. - * This method is called on such unprocessed exchanges and the implementation of this method - * will arrange for the request, corresponding to this exchange, to be retried afresh on a - * new connection. - */ - void closeAsUnprocessed() { - try { - // We arrange for the request to be retried on a new connection as allowed by the RFC-9113 - markUnprocessedByPeer(); - this.errorRef.compareAndSet(null, new IOException("request not processed by peer")); - if (debug.on()) { - debug.log("closing " + this.request + " as unprocessed by peer"); - } - // close the exchange and complete the response CF exceptionally - close(); - completeResponseExceptionally(this.errorRef.get()); - } finally { - // decrementStreamsCount isn't really needed but we do it to make sure - // the log messages, where these counts/states get reported, show the accurate state. - connection.decrementStreamsCount(streamid); - } + return connection.dbgString() + "/Stream("+streamid+")"; } private class HeadersConsumer extends ValidatingHeadersConsumer implements DecodingCallback { diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java b/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java index a30c21a7a7d..1df81640345 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/WindowController.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2016, 2024, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2016, 2018, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -100,16 +100,13 @@ void removeStream(int streamid) { controllerLock.lock(); try { Integer old = streams.remove(streamid); - // A client initiated stream might be closed (as unprocessed, due to a - // GOAWAY received on the connection) even before the stream is - // registered with this WindowController instance (when sending out request headers). - // Thus, for client initiated streams, we don't enforce the presence of the - // stream in the registered "streams" map. - + // Odd stream numbers (client streams) should have been registered. // Even stream numbers (server streams - aka Push Streams) should // not be registered final boolean isClientStream = (streamid & 0x1) == 1; - if (old != null && !isClientStream) { + if (old == null && isClientStream) { + throw new InternalError("Expected entry for streamid: " + streamid); + } else if (old != null && !isClientStream) { throw new InternalError("Unexpected entry for streamid: " + streamid); } } finally { diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java b/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java index 662e5dd98b3..45ec7553f5a 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/frame/GoAwayFrame.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2018, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -57,9 +57,7 @@ int length() { @Override public String toString() { - return super.toString() - + " lastStreamId=" + lastStream - + ", Debugdata: " + new String(debugData, UTF_8); + return super.toString() + " Debugdata: " + new String(debugData, UTF_8); } public int getLastStream() { diff --git a/test/jdk/java/net/httpclient/http2/H2GoAwayTest.java b/test/jdk/java/net/httpclient/http2/H2GoAwayTest.java deleted file mode 100644 index 755bb2e16cc..00000000000 --- a/test/jdk/java/net/httpclient/http2/H2GoAwayTest.java +++ /dev/null @@ -1,336 +0,0 @@ -/* - * Copyright (c) 2024, Oracle and/or its affiliates. All rights reserved. - * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. - * - * This code is free software; you can redistribute it and/or modify it - * under the terms of the GNU General Public License version 2 only, as - * published by the Free Software Foundation. - * - * This code is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License - * version 2 for more details (a copy is included in the LICENSE file that - * accompanied this code). - * - * You should have received a copy of the GNU General Public License version - * 2 along with this work; if not, write to the Free Software Foundation, - * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. - * - * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA - * or visit www.oracle.com if you need additional information or have any - * questions. - */ - -import java.io.IOException; -import java.io.OutputStream; -import java.net.URI; -import java.net.http.HttpClient; -import java.net.http.HttpRequest; -import java.net.http.HttpResponse; -import java.net.http.HttpResponse.BodyHandlers; -import java.util.ArrayList; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; - -import javax.net.ssl.SSLContext; - -import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestExchange; -import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestHandler; -import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestServer; -import jdk.test.lib.net.SimpleSSLContext; -import jdk.test.lib.net.URIBuilder; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import static java.net.http.HttpClient.Version.HTTP_2; -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.fail; - -/* - * @test - * @bug 8335181 - * @summary verify that the HttpClient correctly handles incoming GOAWAY frames and - * retries any unprocessed requests on a new connection - * @library /test/lib /test/jdk/java/net/httpclient/lib - * @build jdk.httpclient.test.lib.common.HttpServerAdapters - * jdk.test.lib.net.SimpleSSLContext - * @run junit H2GoAwayTest - */ -public class H2GoAwayTest { - private static final String REQ_PATH = "/test"; - private static HttpTestServer server; - private static String REQ_URI_BASE; - private static SSLContext sslCtx; - - @BeforeAll - static void beforeAll() throws Exception { - sslCtx = new SimpleSSLContext().get(); - assertNotNull(sslCtx, "SSLContext couldn't be created"); - server = HttpTestServer.create(HTTP_2, sslCtx); - server.addHandler(new Handler(), REQ_PATH); - server.start(); - System.out.println("Server started at " + server.getAddress()); - REQ_URI_BASE = URIBuilder.newBuilder().scheme("https") - .loopback() - .port(server.getAddress().getPort()) - .path(REQ_PATH) - .build().toString(); - } - - @AfterAll - static void afterAll() { - if (server != null) { - System.out.println("Stopping server at " + server.getAddress()); - server.stop(); - } - } - - /** - * Verifies that when several requests are sent using send() and the server - * connection is configured to send a GOAWAY after processing only a few requests, then - * the remaining requests are retried on a different connection - */ - @Test - public void testSequential() throws Exception { - final LimitedPerConnRequestApprover reqApprover = new LimitedPerConnRequestApprover(); - server.setRequestApprover(reqApprover::allowNewRequest); - try (final HttpClient client = HttpClient.newBuilder().version(HTTP_2) - .sslContext(sslCtx).build()) { - final String[] reqMethods = {"HEAD", "GET", "POST"}; - for (final String reqMethod : reqMethods) { - final int numReqs = LimitedPerConnRequestApprover.MAX_REQS_PER_CONN + 3; - final Set connectionKeys = new LinkedHashSet<>(); - for (int i = 1; i <= numReqs; i++) { - final URI reqURI = new URI(REQ_URI_BASE + "?seq&" + reqMethod + "=" + i); - final HttpRequest req = HttpRequest.newBuilder() - .uri(reqURI) - .method(reqMethod, HttpRequest.BodyPublishers.noBody()) - .build(); - System.out.println("initiating request " + req); - final HttpResponse resp = client.send(req, BodyHandlers.ofString()); - final String respBody = resp.body(); - System.out.println("received response: " + respBody); - assertEquals(200, resp.statusCode(), - "unexpected status code for request " + resp.request()); - // response body is the logical key of the connection on which the - // request was handled - connectionKeys.add(respBody); - } - System.out.println("connections involved in handling the requests: " - + connectionKeys); - // all requests have finished, we now just do a basic check that - // more than one connection was involved in processing these requests - assertEquals(2, connectionKeys.size(), - "unexpected number of connections " + connectionKeys); - } - } finally { - server.setRequestApprover(null); // reset - } - } - - /** - * Verifies that when a server responds with a GOAWAY and then never processes the new retried - * requests on a new connection too, then the application code receives the request failure. - * This tests the send() API of the HttpClient. - */ - @Test - public void testUnprocessedRaisesException() throws Exception { - try (final HttpClient client = HttpClient.newBuilder().version(HTTP_2) - .sslContext(sslCtx).build()) { - final Random random = new Random(); - final String[] reqMethods = {"HEAD", "GET", "POST"}; - for (final String reqMethod : reqMethods) { - final int maxAllowedReqs = 2; - final int numReqs = maxAllowedReqs + 3; // 3 more requests than max allowed - // configure the approver - final LimitedRequestApprover reqApprover = new LimitedRequestApprover(maxAllowedReqs); - server.setRequestApprover(reqApprover::allowNewRequest); - try { - int numSuccess = 0; - int numFailed = 0; - for (int i = 1; i <= numReqs; i++) { - final String reqQueryPart = "?sync&" + reqMethod + "=" + i; - final URI reqURI = new URI(REQ_URI_BASE + reqQueryPart); - final HttpRequest req = HttpRequest.newBuilder() - .uri(reqURI) - .method(reqMethod, HttpRequest.BodyPublishers.noBody()) - .build(); - System.out.println("initiating request " + req); - if (i <= maxAllowedReqs) { - // expected to successfully complete - numSuccess++; - final HttpResponse resp = client.send(req, BodyHandlers.ofString()); - final String respBody = resp.body(); - System.out.println("received response: " + respBody); - assertEquals(200, resp.statusCode(), - "unexpected status code for request " + resp.request()); - } else { - // expected to fail as unprocessed - try { - final HttpResponse resp = client.send(req, BodyHandlers.ofString()); - fail("Request was expected to fail as unprocessed," - + " but got response: " + resp.body() + ", status code: " - + resp.statusCode()); - } catch (IOException ioe) { - // verify it failed for the right reason - if (ioe.getMessage() == null - || !ioe.getMessage().contains("request not processed by peer")) { - // propagate the original failure - throw ioe; - } - numFailed++; // failed due to right reason - System.out.println("received expected failure: " + ioe - + ", for request " + reqURI); - } - } - } - // verify the correct number of requests succeeded/failed - assertEquals(maxAllowedReqs, numSuccess, "unexpected number of requests succeeded"); - assertEquals((numReqs - maxAllowedReqs), numFailed, "unexpected number of requests failed"); - } finally { - server.setRequestApprover(null); // reset - } - } - } - } - - /** - * Verifies that when a server responds with a GOAWAY and then never processes the new retried - * requests on a new connection too, then the application code receives the request failure. - * This tests the sendAsync() API of the HttpClient. - */ - @Test - public void testUnprocessedRaisesExceptionAsync() throws Throwable { - try (final HttpClient client = HttpClient.newBuilder().version(HTTP_2) - .sslContext(sslCtx).build()) { - final Random random = new Random(); - final String[] reqMethods = {"HEAD", "GET", "POST"}; - for (final String reqMethod : reqMethods) { - final int maxAllowedReqs = 2; - final int numReqs = maxAllowedReqs + 3; // 3 more requests than max allowed - // configure the approver - final LimitedRequestApprover reqApprover = new LimitedRequestApprover(maxAllowedReqs); - server.setRequestApprover(reqApprover::allowNewRequest); - try { - final List>> futures = new ArrayList<>(); - for (int i = 1; i <= numReqs; i++) { - final URI reqURI = new URI(REQ_URI_BASE + "?async&" + reqMethod + "=" + i); - final HttpRequest req = HttpRequest.newBuilder() - .uri(reqURI) - .method(reqMethod, HttpRequest.BodyPublishers.noBody()) - .build(); - System.out.println("initiating request " + req); - final Future> f = client.sendAsync(req, BodyHandlers.ofString()); - futures.add(f); - } - // wait for responses - int numFailed = 0; - int numSuccess = 0; - for (int i = 1; i <= numReqs; i++) { - final String reqQueryPart = "?async&" + reqMethod + "=" + i; - try { - System.out.println("waiting response of request " - + REQ_URI_BASE + reqQueryPart); - final HttpResponse resp = futures.get(i - 1).get(); - numSuccess++; - final String respBody = resp.body(); - System.out.println("request: " + resp.request() - + ", received response: " + respBody); - assertEquals(200, resp.statusCode(), - "unexpected status code for request " + resp.request()); - } catch (ExecutionException ee) { - final Throwable cause = ee.getCause(); - if (!(cause instanceof IOException ioe)) { - throw cause; - } - // verify it failed for the right reason - if (ioe.getMessage() == null - || !ioe.getMessage().contains("request not processed by peer")) { - // propagate the original failure - throw ioe; - } - numFailed++; // failed due to the right reason - System.out.println("received expected failure: " + ioe - + ", for request " + REQ_URI_BASE + reqQueryPart); - } - } - // verify the correct number of requests succeeded/failed - assertEquals(maxAllowedReqs, numSuccess, "unexpected number of requests succeeded"); - assertEquals((numReqs - maxAllowedReqs), numFailed, "unexpected number of requests failed"); - } finally { - server.setRequestApprover(null); // reset - } - } - } - } - - // only allows fixed number of requests, irrespective of which server connection handles - // it. requests that are rejected will either be sent a GOAWAY on the connection - // or a RST_FRAME with a REFUSED_STREAM on the stream - private static final class LimitedRequestApprover { - private final int maxAllowedReqs; - private final AtomicInteger numApproved = new AtomicInteger(); - - private LimitedRequestApprover(final int maxAllowedReqs) { - this.maxAllowedReqs = maxAllowedReqs; - } - - public boolean allowNewRequest(final String serverConnKey) { - final int approved = numApproved.incrementAndGet(); - return approved <= maxAllowedReqs; - } - } - - // allows a certain number of requests per server connection. - // requests that are rejected will either be sent a GOAWAY on the connection - // or a RST_FRAME with a REFUSED_STREAM on the stream - private static final class LimitedPerConnRequestApprover { - private static final int MAX_REQS_PER_CONN = 6; - private final Map numApproved = - new ConcurrentHashMap<>(); - private final Map numDisapproved = - new ConcurrentHashMap<>(); - - public boolean allowNewRequest(final String serverConnKey) { - final AtomicInteger approved = numApproved.computeIfAbsent(serverConnKey, - (k) -> new AtomicInteger()); - int curr = approved.get(); - while (curr < MAX_REQS_PER_CONN) { - if (approved.compareAndSet(curr, curr + 1)) { - return true; // new request allowed - } - curr = approved.get(); - } - final AtomicInteger disapproved = numDisapproved.computeIfAbsent(serverConnKey, - (k) -> new AtomicInteger()); - final int numUnprocessed = disapproved.incrementAndGet(); - System.out.println(approved.get() + " processed, " - + numUnprocessed + " unprocessed requests on connection " + serverConnKey); - return false; - } - } - - private static final class Handler implements HttpTestHandler { - - @Override - public void handle(final HttpTestExchange exchange) throws IOException { - final String connectionKey = exchange.getConnectionKey(); - System.out.println("responding to request: " + exchange.getRequestURI() - + " on connection " + connectionKey); - final byte[] response = connectionKey.getBytes(UTF_8); - exchange.sendResponseHeaders(200, response.length); - try (final OutputStream os = exchange.getResponseBody()) { - os.write(response); - } - } - } -} diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java index b91952ba410..98da7d5533f 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/common/HttpServerAdapters.java @@ -58,7 +58,6 @@ import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; -import java.util.function.Predicate; import java.util.logging.Level; import java.util.logging.Logger; import java.util.stream.Collectors; @@ -241,7 +240,6 @@ public static abstract class HttpTestExchange implements AutoCloseable { public abstract void close(); public abstract InetSocketAddress getRemoteAddress(); public abstract InetSocketAddress getLocalAddress(); - public abstract String getConnectionKey(); public void serverPush(URI uri, HttpHeaders headers, byte[] body) { ByteArrayInputStream bais = new ByteArrayInputStream(body); serverPush(uri, headers, bais); @@ -256,7 +254,7 @@ public static HttpTestExchange of(HttpExchange exchange) { return new Http1TestExchange(exchange); } public static HttpTestExchange of(Http2TestExchange exchange) { - return new H2ExchangeImpl(exchange); + return new Http2TestExchangeImpl(exchange); } abstract void doFilter(Filter.Chain chain) throws IOException; @@ -312,21 +310,15 @@ public InetSocketAddress getLocalAddress() { public URI getRequestURI() { return exchange.getRequestURI(); } @Override public String getRequestMethod() { return exchange.getRequestMethod(); } - - @Override - public String getConnectionKey() { - return exchange.getLocalAddress() + "->" + exchange.getRemoteAddress(); - } - @Override public String toString() { return this.getClass().getSimpleName() + ": " + exchange.toString(); } } - private static final class H2ExchangeImpl extends HttpTestExchange { + private static final class Http2TestExchangeImpl extends HttpTestExchange { private final Http2TestExchange exchange; - H2ExchangeImpl(Http2TestExchange exch) { + Http2TestExchangeImpl(Http2TestExchange exch) { this.exchange = exch; } @Override @@ -379,11 +371,6 @@ public InetSocketAddress getLocalAddress() { return exchange.getLocalAddress(); } - @Override - public String getConnectionKey() { - return exchange.getConnectionKey(); - } - @Override public URI getRequestURI() { return exchange.getRequestURI(); } @Override @@ -729,7 +716,6 @@ static void enableLogging() { public abstract HttpTestContext addHandler(HttpTestHandler handler, String root); public abstract InetSocketAddress getAddress(); public abstract Version getVersion(); - public abstract void setRequestApprover(final Predicate approver); public String serverAuthority() { InetSocketAddress address = getAddress(); @@ -878,11 +864,6 @@ public InetSocketAddress getAddress() { impl.getAddress().getPort()); } public Version getVersion() { return Version.HTTP_1_1; } - - @Override - public void setRequestApprover(final Predicate approver) { - throw new UnsupportedOperationException("not supported"); - } } private static class Http1TestContext extends HttpTestContext { @@ -934,11 +915,6 @@ public InetSocketAddress getAddress() { impl.getAddress().getPort()); } public Version getVersion() { return Version.HTTP_2; } - - @Override - public void setRequestApprover(final Predicate approver) { - this.impl.setRequestApprover(approver); - } } private static class Http2TestContext diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java index 828c939f53f..1a8b5d92af5 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2017, 2024, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2017, 2023, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -84,10 +84,4 @@ default void sendFrames(List frames) throws IOException { * It may also complete exceptionally */ CompletableFuture sendPing(); - - /** - * {@return the identification of the connection on which this exchange is being - * processed} - */ - String getConnectionKey(); } diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java index c867b8bf040..0abcd51af63 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java @@ -227,11 +227,6 @@ public void serverPush(URI uri, HttpHeaders headers, InputStream content) { } } - @Override - public String getConnectionKey() { - return conn.connectionKey(); - } - private boolean isHeadRequest() { return HEAD.equalsIgnoreCase(getRequestMethod()); } diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java index ca7e7d3dc30..ffa05d5c11c 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServer.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2023, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -32,8 +32,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Predicate; - import javax.net.ServerSocketFactory; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLParameters; @@ -61,8 +59,6 @@ public class Http2TestServer implements AutoCloseable { final Set connections; final Properties properties; final String name; - // request approver which takes the server connection key as the input - private volatile Predicate newRequestApprover; private static ThreadFactory defaultThreadFac = (Runnable r) -> { @@ -289,14 +285,6 @@ public String serverName() { return serverName; } - public void setRequestApprover(final Predicate approver) { - this.newRequestApprover = approver; - } - - Predicate getRequestApprover() { - return this.newRequestApprover; - } - private synchronized void putConnection(InetSocketAddress addr, Http2TestServerConnection c) { if (!stopping) connections.add(c); diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java index 40cc1332c76..b087a037222 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java @@ -77,20 +77,15 @@ import java.util.Optional; import java.util.Properties; import java.util.Random; -import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BiPredicate; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import java.util.function.Consumer; -import java.util.function.Predicate; import static java.nio.charset.StandardCharsets.ISO_8859_1; import static java.nio.charset.StandardCharsets.UTF_8; -import static jdk.internal.net.http.frame.ErrorFrame.REFUSED_STREAM; import static jdk.internal.net.http.frame.SettingsFrame.DEFAULT_MAX_FRAME_SIZE; import static jdk.internal.net.http.frame.SettingsFrame.HEADER_TABLE_SIZE; @@ -120,10 +115,6 @@ public class Http2TestServerConnection { volatile boolean stopping; volatile int nextPushStreamId = 2; ConcurrentLinkedQueue pings = new ConcurrentLinkedQueue<>(); - // the max stream id of a processed H2 request. -1 implies none were processed. - private final AtomicInteger maxProcessedRequestStreamId = new AtomicInteger(-1); - // the stream id that was sent in a GOAWAY frame. -1 implies no GOAWAY frame was sent. - private final AtomicInteger goAwayRequestStreamId = new AtomicInteger(-1); final static ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0); final static byte[] EMPTY_BARRAY = new byte[0]; @@ -248,29 +239,11 @@ CompletableFuture sendPing() { return ping.response(); } - private void sendGoAway(final int error) throws IOException { - int maxProcessedStreamId = maxProcessedRequestStreamId.get(); - if (maxProcessedStreamId == -1) { - maxProcessedStreamId = 0; - } - boolean send = false; - int currentGoAwayReqStrmId = goAwayRequestStreamId.get(); - // update the last processed stream id and send a goaway frame if the new last processed - // stream id is lesser than the last processed stream id sent in - // a previous goaway frame (if any) - while (currentGoAwayReqStrmId == -1 || maxProcessedStreamId < currentGoAwayReqStrmId) { - if (goAwayRequestStreamId.compareAndSet(currentGoAwayReqStrmId, maxProcessedStreamId)) { - send = true; - break; - } - currentGoAwayReqStrmId = goAwayRequestStreamId.get(); - } - if (!send) { - return; - } - final GoAwayFrame frame = new GoAwayFrame(maxProcessedStreamId, error); - outputQ.put(frame); - System.err.println("Sending GOAWAY frame " + frame + " from server connection " + this); + void goAway(int error) throws IOException { + int laststream = nextstream >= 3 ? nextstream - 2 : 1; + + GoAwayFrame go = new GoAwayFrame(laststream, error); + outputQ.put(go); } /** @@ -363,9 +336,8 @@ void close(int error) { q.orderlyClose(); }); try { - if (error != -1) { - sendGoAway(error); - } + if (error != -1) + goAway(error); outputQ.orderlyClose(); socket.close(); } catch (Exception e) { @@ -647,14 +619,6 @@ void createPrimordialStream(Http1InitialRequest request) throws IOException { path = path + "?" + uri.getRawQuery(); headersBuilder.setHeader(":path", path); - // skip processing the request if configured to do so - final String connKey = connectionKey(); - if (!shouldProcessNewHTTPRequest(connKey)) { - System.err.println("Rejecting primordial stream 1 and sending GOAWAY" + - " on server connection " + connKey + ", for request: " + path); - sendGoAway(ErrorFrame.NO_ERROR); - return; - } Queue q = new Queue(sentinel); byte[] body = getRequestBody(request); addHeaders(getHeaders(request.headers), headersBuilder); @@ -663,24 +627,11 @@ void createPrimordialStream(Http1InitialRequest request) throws IOException { addRequestBodyToQueue(body, q); streams.put(1, q); - maxProcessedRequestStreamId.set(1); exec.submit(() -> { handleRequest(headers, q, 1, true /*complete request has been read*/); }); } - private boolean shouldProcessNewHTTPRequest(final String serverConnKey) { - final Predicate approver = this.server.getRequestApprover(); - if (approver == null) { - return true; // process the request - } - return approver.test(serverConnKey); - } - - final String connectionKey() { - return this.server.getAddress() + "->" + this.socket.getRemoteSocketAddress(); - } - // all other streams created here @SuppressWarnings({"rawtypes","unchecked"}) void createStream(HeaderFrame frame) throws IOException { @@ -688,7 +639,7 @@ void createStream(HeaderFrame frame) throws IOException { frames.add(frame); int streamid = frame.streamid(); if (streamid != nextstream) { - throw new IOException("unexpected stream id: " + streamid); + throw new IOException("unexpected stream id"); } nextstream += 2; @@ -719,30 +670,12 @@ void createStream(HeaderFrame frame) throws IOException { throw new IOException("Unexpected Upgrade in headers:" + headers); } disallowedHeader = headers.firstValue("HTTP2-Settings"); - if (disallowedHeader.isPresent()) { + if (disallowedHeader.isPresent()) throw new IOException("Unexpected HTTP2-Settings in headers:" + headers); - } - // skip processing the request if the server is configured to do so - final String connKey = connectionKey(); - final String path = headers.firstValue(":path").orElse(""); - if (!shouldProcessNewHTTPRequest(connKey)) { - System.err.println("Rejecting stream " + streamid - + " and sending GOAWAY on server connection " - + connKey + ", for request: " + path); - sendGoAway(ErrorFrame.NO_ERROR); - return; - } + Queue q = new Queue(sentinel); streams.put(streamid, q); - // keep track of the largest request id that we have processed - int currentLargest = maxProcessedRequestStreamId.get(); - while (streamid > currentLargest) { - if (maxProcessedRequestStreamId.compareAndSet(currentLargest, streamid)) { - break; - } - currentLargest = maxProcessedRequestStreamId.get(); - } exec.submit(() -> { handleRequest(headers, q, streamid, endStreamReceived); }); @@ -845,8 +778,6 @@ void readLoop() { while (!stopping) { Http2Frame frame = readFrameImpl(); if (frame == null) { - System.err.println("EOF reached on connection " + connectionKey() - + ", will no longer accept incoming frames"); closeIncoming(); return; } @@ -870,17 +801,6 @@ void readLoop() { // TODO: close connection continue; } else { - final int streamId = frame.streamid(); - final int finalProcessedStreamId = goAwayRequestStreamId.get(); - // if we already sent a goaway, then don't create new streams with - // higher stream ids. - if (finalProcessedStreamId != -1 && streamId > finalProcessedStreamId) { - System.err.println(connectionKey() + " resetting stream " + streamId - + " as REFUSED_STREAM"); - final ResetFrame rst = new ResetFrame(streamId, REFUSED_STREAM); - outputQ.put(rst); - continue; - } createStream((HeadersFrame) frame); } } else { From 0857043a4dd88c4629994c76404d970c1fdadb1d Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Fri, 21 Feb 2025 13:09:03 -0800 Subject: [PATCH 7/9] Revert "Backport fc98998627443d6e73ac70661f47f48b30525712" This reverts commit 899abf76e7a381abdfbc30f08a6a5678c0a8065d. --- .../jdk/internal/net/http/HttpClientImpl.java | 35 +- .../jdk/internal/net/http/MultiExchange.java | 39 +- .../httpclient/HttpGetInCancelledFuture.java | 391 ------------------ .../java/net/httpclient/ReferenceTracker.java | 12 +- 4 files changed, 24 insertions(+), 453 deletions(-) delete mode 100644 test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java b/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java index 9a8b901df7d..185f100fd1a 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/HttpClientImpl.java @@ -282,25 +282,23 @@ public String toString() { } } - static CompletableFuture registerPending(PendingRequest pending, CompletableFuture res) { + static void registerPending(PendingRequest pending) { // shortcut if cf is already completed: no need to go through the trouble of // registering it - if (pending.cf.isDone()) return res; + if (pending.cf.isDone()) return; var client = pending.client; var cf = pending.cf; var id = pending.id; boolean added = client.pendingRequests.add(pending); // this may immediately remove `pending` from the set is the cf is already completed - var ref = res.whenComplete((r,t) -> client.pendingRequests.remove(pending)); - pending.ref = ref; + pending.ref = cf.whenComplete((r,t) -> client.pendingRequests.remove(pending)); assert added : "request %d was already added".formatted(id); // should not happen, unless the selector manager has already // exited abnormally if (client.selmgr.isClosed()) { pending.abort(client.selmgr.selectorClosedException()); } - return ref; } static void abortPendingRequests(HttpClientImpl client, Throwable reason) { @@ -933,9 +931,8 @@ private void debugCompleted(String tag, long startNanos, HttpRequest req) { cf = sendAsync(req, responseHandler, null, null); return cf.get(); } catch (InterruptedException ie) { - if (cf != null) { + if (cf != null ) cf.cancel(true); - } throw ie; } catch (ExecutionException e) { final Throwable throwable = e.getCause(); @@ -1057,23 +1054,19 @@ private void debugCompleted(String tag, long startNanos, HttpRequest req) { (b,t) -> debugCompleted("ClientImpl (async)", start, userRequest)); } + // makes sure that any dependent actions happen in the CF default + // executor. This is only needed for sendAsync(...), when + // exchangeExecutor is non-null. + if (exchangeExecutor != null) { + res = res.whenCompleteAsync((r, t) -> { /* do nothing */}, ASYNC_POOL); + } + // The mexCf is the Cf we need to abort if the SelectorManager thread // is aborted. PendingRequest pending = new PendingRequest(id, requestImpl, mexCf, mex, this); - res = registerPending(pending, res); - - if (exchangeExecutor != null) { - // makes sure that any dependent actions happen in the CF default - // executor. This is only needed for sendAsync(...), when - // exchangeExecutor is non-null. - return res.isDone() ? res - : res.whenCompleteAsync((r, t) -> { /* do nothing */}, ASYNC_POOL); - } else { - // make a defensive copy that can be safely canceled - // by the caller - return res.isDone() ? res : res.copy(); - } - } catch (Throwable t) { + registerPending(pending); + return res; + } catch(Throwable t) { requestUnreference(); debugCompleted("ClientImpl (async)", start, userRequest); throw t; diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java index b3bc83e93de..3d1f9255565 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/MultiExchange.java @@ -91,7 +91,7 @@ class MultiExchange implements Cancelable { Exchange previous; volatile Throwable retryCause; volatile boolean expiredOnce; - volatile HttpResponse response; + volatile HttpResponse response = null; // Maximum number of times a request will be retried/redirected // for any reason @@ -279,19 +279,11 @@ public void cancel(IOException cause) { @Override public boolean cancel(boolean mayInterruptIfRunning) { boolean cancelled = this.cancelled; - boolean firstCancel = false; if (!cancelled && mayInterruptIfRunning) { if (interrupted.get() == null) { - firstCancel = interrupted.compareAndSet(null, + interrupted.compareAndSet(null, new CancellationException("Request cancelled")); } - if (debug.on()) { - if (firstCancel) { - debug.log("multi exchange recording: " + interrupted.get()); - } else { - debug.log("multi exchange recorded: " + interrupted.get()); - } - } this.cancelled = true; var exchange = getExchange(); if (exchange != null) { @@ -373,30 +365,17 @@ private CompletableFuture> handleNoBody(Response r, Exchange }).exceptionallyCompose(this::whenCancelled); } - // returns a CancellationExcpetion that wraps the given cause - // if cancel(boolean) was called, the given cause otherwise - private Throwable wrapIfCancelled(Throwable cause) { - CancellationException interrupt = interrupted.get(); - if (interrupt == null) return cause; - - var cancel = new CancellationException(interrupt.getMessage()); - // preserve the stack trace of the original exception to - // show where the call to cancel(boolean) came from - cancel.setStackTrace(interrupt.getStackTrace()); - cancel.initCause(Utils.getCancelCause(cause)); - return cancel; - } - - // if the request failed because the multi exchange was cancelled, - // make sure the reported exception is wrapped in CancellationException private CompletableFuture> whenCancelled(Throwable t) { - var x = wrapIfCancelled(t); - if (x instanceof CancellationException) { + CancellationException x = interrupted.get(); + if (x != null) { + // make sure to fail with CancellationException if cancel(true) + // was called. + t = x.initCause(Utils.getCancelCause(t)); if (debug.on()) { - debug.log("MultiExchange interrupted with: " + x.getCause()); + debug.log("MultiExchange interrupted with: " + t.getCause()); } } - return MinimalFuture.failedFuture(x); + return MinimalFuture.failedFuture(t); } static class NullSubscription implements Flow.Subscription { diff --git a/test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java b/test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java deleted file mode 100644 index 5efc50138d6..00000000000 --- a/test/jdk/java/net/httpclient/HttpGetInCancelledFuture.java +++ /dev/null @@ -1,391 +0,0 @@ -/* - * Copyright (c) 2023, Oracle and/or its affiliates. All rights reserved. - * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. - * - * This code is free software; you can redistribute it and/or modify it - * under the terms of the GNU General Public License version 2 only, as - * published by the Free Software Foundation. - * - * This code is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License - * version 2 for more details (a copy is included in the LICENSE file that - * accompanied this code). - * - * You should have received a copy of the GNU General Public License version - * 2 along with this work; if not, write to the Free Software Foundation, - * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. - * - * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA - * or visit www.oracle.com if you need additional information or have any - * questions. - */ - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.ServerSocket; -import java.net.URI; -import java.net.http.HttpClient; -import java.net.http.HttpClient.Version; -import java.net.http.HttpRequest; -import java.net.http.HttpResponse; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.Future; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import jdk.internal.net.http.common.OperationTrackers.Tracker; -import jdk.test.lib.net.SimpleSSLContext; -import jdk.test.lib.net.URIBuilder; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; - -/* - * @test - * @bug 8316580 - * @library /test/lib - * @run junit/othervm -Djdk.tracePinnedThreads=full - * -DuseReferenceTracker=false - * HttpGetInCancelledFuture - * @run junit/othervm -Djdk.tracePinnedThreads=full - * -DuseReferenceTracker=true - * HttpGetInCancelledFuture - * @summary This test verifies that cancelling a future that - * does an HTTP request using the HttpClient doesn't cause - * HttpClient::close to block forever. - */ -public class HttpGetInCancelledFuture { - - static final boolean useTracker = Boolean.getBoolean("useReferenceTracker"); - - static final class TestException extends RuntimeException { - public TestException(String message, Throwable cause) { - super(message, cause); - } - } - - static ReferenceTracker TRACKER = ReferenceTracker.INSTANCE; - - HttpClient makeClient(URI uri, Version version, Executor executor) { - var builder = HttpClient.newBuilder(); - if (uri.getScheme().equalsIgnoreCase("https")) { - try { - builder.sslContext(new SimpleSSLContext().get()); - } catch (IOException io) { - throw new UncheckedIOException(io); - } - } - return builder.connectTimeout(Duration.ofSeconds(1)) - .executor(executor) - .version(version) - .build(); - } - - record TestCase(String url, int reqCount, Version version) {} - // A server that doesn't accept - static volatile ServerSocket NOT_ACCEPTING; - - static List parameters() { - ServerSocket ss = NOT_ACCEPTING; - if (ss == null) { - synchronized (HttpGetInCancelledFuture.class) { - if ((ss = NOT_ACCEPTING) == null) { - try { - ss = new ServerSocket(); - var loopback = InetAddress.getLoopbackAddress(); - ss.bind(new InetSocketAddress(loopback, 0), 10); - NOT_ACCEPTING = ss; - } catch (IOException io) { - throw new UncheckedIOException(io); - } - } - } - } - URI http = URIBuilder.newBuilder() - .loopback() - .scheme("http") - .port(ss.getLocalPort()) - .path("/not-accepting/") - .buildUnchecked(); - URI https = URIBuilder.newBuilder() - .loopback() - .scheme("https") - .port(ss.getLocalPort()) - .path("/not-accepting/") - .buildUnchecked(); - // use all HTTP versions, without and with TLS - return List.of( - new TestCase(http.toString(), 200, Version.HTTP_2), - new TestCase(http.toString(), 200, Version.HTTP_1_1), - new TestCase(https.toString(), 200, Version.HTTP_2), - new TestCase(https.toString(), 200, Version.HTTP_1_1) - ); - } - - @ParameterizedTest - @MethodSource("parameters") - void runTest(TestCase test) { - System.out.println("Testing with: " + test); - runTest(test.url, test.reqCount, test.version); - } - - static class TestTaskScope implements AutoCloseable { - final ExecutorService pool = new ForkJoinPool(); - final Map, Future> tasks = new ConcurrentHashMap<>(); - final AtomicReference failed = new AtomicReference<>(); - - class Task implements Callable { - final Callable task; - final CompletableFuture cf = new CompletableFuture<>(); - Task(Callable task) { - this.task = task; - } - @Override - public T call() throws Exception { - try { - var res = task.call(); - cf.complete(res); - return res; - } catch (Throwable t) { - cf.completeExceptionally(t); - throw t; - } - } - CompletableFuture cf() { - return cf; - } - } - - - static class ShutdownOnFailure extends TestTaskScope { - public ShutdownOnFailure() {} - - @Override - protected void completed(Task task, T result, Throwable throwable) { - super.completed(task, result, throwable); - if (throwable != null) { - if (failed.get() == null) { - ExecutionException ex = throwable instanceof ExecutionException x - ? x : new ExecutionException(throwable); - failed.compareAndSet(null, ex); - } - tasks.entrySet().forEach(this::cancel); - } - } - - void cancel(Map.Entry, Future> entry) { - entry.getValue().cancel(true); - entry.getKey().cf().cancel(true); - tasks.remove(entry.getKey(), entry.getValue()); - } - - @Override - public CompletableFuture fork(Callable callable) { - var ex = failed.get(); - if (ex == null) { - return super.fork(callable); - } // otherwise do nothing - return CompletableFuture.failedFuture(new RejectedExecutionException()); - } - } - - public CompletableFuture fork(Callable callable) { - var task = new Task<>(callable); - var res = pool.submit(task); - tasks.put(task, res); - task.cf.whenComplete((r,t) -> completed(task, r, t)); - return task.cf; - } - - protected void completed(Task task, T result, Throwable throwable) { - tasks.remove(task); - } - - public void join() throws InterruptedException { - try { - var cfs = tasks.keySet().stream() - .map(Task::cf).toArray(CompletableFuture[]::new); - CompletableFuture.allOf(cfs).get(); - } catch (InterruptedException it) { - throw it; - } catch (ExecutionException ex) { - failed.compareAndSet(null, ex); - } - } - - public void throwIfFailed() throws ExecutionException { - ExecutionException x = failed.get(); - if (x != null) throw x; - } - - public void close() { - pool.close(); - } - } - - ExecutorService testExecutor() { - return Executors.newCachedThreadPool(); - } - - void runTest(String url, int reqCount, Version version) { - final var dest = URI.create(url); - try (final var executor = testExecutor()) { - var httpClient = makeClient(dest, version, executor); - TRACKER.track(httpClient); - Tracker tracker = TRACKER.getTracker(httpClient); - Throwable failed = null; - try { - try (final var scope = new TestTaskScope.ShutdownOnFailure()) { - launchAndProcessRequests(scope, httpClient, reqCount, dest); - } finally { - System.out.printf("StructuredTaskScope closed: STARTED=%s, SUCCESS=%s, INTERRUPT=%s, FAILED=%s%n", - STARTED.get(), SUCCESS.get(), INTERRUPT.get(), FAILED.get()); - } - System.out.println("ERROR: Expected TestException not thrown"); - throw new AssertionError("Expected TestException not thrown"); - } catch (TestException x) { - System.out.println("Got expected exception: " + x); - } catch (Throwable t) { - System.out.println("ERROR: Unexpected exception: " + t); - failed = t; - throw t; - } finally { - // we can either use the tracker or call HttpClient::close - if (useTracker) { - // using the tracker depends on GC but will give us some diagnostic - // if some operations are not properly cancelled and prevent the client - // from terminating - httpClient = null; - System.gc(); - System.out.println(TRACKER.diagnose(tracker)); - var error = TRACKER.check(tracker, 10000); - if (error != null) { - if (failed != null) error.addSuppressed(failed); - EXCEPTIONS.forEach(x -> { - System.out.println("FAILED: " + x); - }); - EXCEPTIONS.forEach(x -> { - x.printStackTrace(System.out); - }); - throw error; - } - } else { - // if not all operations terminate, close() will block - // forever and the test will fail in jtreg timeout. - // there will be no diagnostic. - httpClient.close(); - } - System.out.println("HttpClient closed"); - } - } finally { - System.out.println("ThreadExecutor closed"); - } - // not all tasks may have been started before the scope was cancelled - // due to the first connect/timeout exception, but all tasks that started - // must have either succeeded, be interrupted, or failed - assertTrue(STARTED.get() > 0); - assertEquals(STARTED.get(), SUCCESS.get() + INTERRUPT.get() + FAILED.get()); - if (SUCCESS.get() > 0) { - // we don't expect any server to be listening and responding - System.out.println("WARNING: got some unexpected successful responses from " - + "\"" + NOT_ACCEPTING.getLocalSocketAddress() + "\": " + SUCCESS.get()); - } - } - - private void launchAndProcessRequests( - TestTaskScope.ShutdownOnFailure scope, - HttpClient httpClient, - int reqCount, - URI dest) { - for (int counter = 0; counter < reqCount; counter++) { - scope.fork(() -> - getAndCheck(httpClient, dest) - ); - } - try { - scope.join(); - } catch (InterruptedException e) { - throw new AssertionError("scope.join() was interrupted", e); - } - try { - scope.throwIfFailed(); - } catch (ExecutionException e) { - throw new TestException("something threw an exception in StructuredTaskScope", e); - } - } - - final static AtomicLong ID = new AtomicLong(); - final AtomicLong SUCCESS = new AtomicLong(); - final AtomicLong INTERRUPT = new AtomicLong(); - final AtomicLong FAILED = new AtomicLong(); - final AtomicLong STARTED = new AtomicLong(); - final CopyOnWriteArrayList EXCEPTIONS = new CopyOnWriteArrayList<>(); - private String getAndCheck(HttpClient httpClient, URI url) { - STARTED.incrementAndGet(); - final var response = sendRequest(httpClient, url); - String res = response.body(); - int statusCode = response.statusCode(); - assertEquals(200, statusCode); - return res; - } - - private HttpResponse sendRequest(HttpClient httpClient, URI url) { - var id = ID.incrementAndGet(); - try { - var request = HttpRequest.newBuilder(url).GET().build(); - var response = httpClient.send(request, HttpResponse.BodyHandlers.ofString()); - // System.out.println("Got response for " + id + ": " + response); - SUCCESS.incrementAndGet(); - return response; - } catch (InterruptedException e) { - INTERRUPT.incrementAndGet(); - // System.out.println("Got interrupted for " + id + ": " + e); - throw new RuntimeException(e); - } catch (Exception e) { - FAILED.incrementAndGet(); - EXCEPTIONS.add(e); - //System.out.println("Got exception for " + id + ": " + e); - throw new RuntimeException(e); - } - } - - @AfterAll - static void tearDown() { - try { - System.gc(); - var error = TRACKER.check(5000); - if (error != null) throw error; - } finally { - ServerSocket ss; - synchronized (HttpGetInCancelledFuture.class) { - ss = NOT_ACCEPTING; - NOT_ACCEPTING = null; - } - if (ss != null) { - try { - ss.close(); - } catch (IOException io) { - throw new UncheckedIOException(io); - } - } - } - } -} - diff --git a/test/jdk/java/net/httpclient/ReferenceTracker.java b/test/jdk/java/net/httpclient/ReferenceTracker.java index d7e16d01201..51dc1f40537 100644 --- a/test/jdk/java/net/httpclient/ReferenceTracker.java +++ b/test/jdk/java/net/httpclient/ReferenceTracker.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2018, 2023, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2018, 2022, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -254,11 +254,6 @@ public AssertionError check(Tracker tracker, } long duration = Duration.ofNanos(System.nanoTime() - waitStart).toMillis(); if (hasOutstanding.test(tracker)) { - if (i == 0 && waited == 0) { - // we found nothing and didn't wait expecting success, but then found - // something. Respin to make sure we wait. - return check(tracker, graceDelayMs, hasOutstanding, description, printThreads); - } StringBuilder warnings = diagnose(tracker, new StringBuilder(), hasOutstanding); if (hasOutstanding.test(tracker)) { fail = new AssertionError(warnings.toString()); @@ -315,11 +310,6 @@ public AssertionError check(long graceDelayMs, } long duration = Duration.ofNanos(System.nanoTime() - waitStart).toMillis(); if (TRACKERS.stream().anyMatch(hasOutstanding)) { - if (i == 0 && waited == 0) { - // we found nothing and didn't wait expecting success, but then found - // something. Respin to make sure we wait. - return check(graceDelayMs, hasOutstanding, description, printThreads); - } StringBuilder warnings = diagnose(new StringBuilder(), hasOutstanding); addSummary(warnings); if (TRACKERS.stream().anyMatch(hasOutstanding)) { From fe8037e701dc1fa2a7d05fa0978721756fba4e11 Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Fri, 21 Feb 2025 13:36:49 -0800 Subject: [PATCH 8/9] Backport b0ac633b2d0076d64b463b2a6ce19abf6b12c50f --- .../internal/net/http/Http2ClientImpl.java | 8 +- .../internal/net/http/Http2Connection.java | 48 ++- .../classes/jdk/internal/net/http/Stream.java | 65 +++- .../internal/net/http/WindowUpdateSender.java | 142 ++++++- .../net/http/frame/FramesDecoder.java | 19 +- .../net/http/frame/SettingsFrame.java | 7 +- .../share/classes/module-info.java | 11 +- .../net/httpclient/GZIPInputStreamTest.java | 2 +- .../net/httpclient/ProxySelectorTest.java | 2 +- .../http2/ConnectionFlowControlTest.java | 365 ++++++++++++++++++ .../http2/StreamFlowControlTest.java | 342 ++++++++++++++++ .../test/lib/http2/BodyOutputStream.java | 36 +- .../lib/http2/Http2TestServerConnection.java | 12 +- 13 files changed, 1022 insertions(+), 37 deletions(-) create mode 100644 test/jdk/java/net/httpclient/http2/ConnectionFlowControlTest.java create mode 100644 test/jdk/java/net/httpclient/http2/StreamFlowControlTest.java diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/Http2ClientImpl.java b/src/java.net.http/share/classes/jdk/internal/net/http/Http2ClientImpl.java index a302006cae3..6f5ec1016fb 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/Http2ClientImpl.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/Http2ClientImpl.java @@ -40,6 +40,8 @@ import jdk.internal.net.http.common.MinimalFuture; import jdk.internal.net.http.common.Utils; import jdk.internal.net.http.frame.SettingsFrame; + +import static jdk.internal.net.http.frame.SettingsFrame.INITIAL_CONNECTION_WINDOW_SIZE; import static jdk.internal.net.http.frame.SettingsFrame.INITIAL_WINDOW_SIZE; import static jdk.internal.net.http.frame.SettingsFrame.ENABLE_PUSH; import static jdk.internal.net.http.frame.SettingsFrame.HEADER_TABLE_SIZE; @@ -289,9 +291,13 @@ int getConnectionWindowSize(SettingsFrame clientSettings) { // and the connection window size. int defaultValue = Math.max(streamWindow, K*K*32); + // The min value is the max between the streamWindow and + // the initial connection window size + int minValue = Math.max(INITIAL_CONNECTION_WINDOW_SIZE, streamWindow); + return getParameter( "jdk.httpclient.connectionWindowSize", - streamWindow, Integer.MAX_VALUE, defaultValue); + minValue, Integer.MAX_VALUE, defaultValue); } /** diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java b/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java index 84c09d6bca6..be575072f2b 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/Http2Connection.java @@ -1064,6 +1064,34 @@ private String checkMaxOrphanedHeadersExceeded(HeaderFrame hf) { return null; } + // This method is called when a DataFrame that was added + // to a Stream::inputQ is later dropped from the queue + // without being consumed. + // + // Before adding a frame to the queue, the Stream calls + // connection.windowUpdater.canBufferUnprocessedBytes(), which + // increases the count of unprocessed bytes in the connection. + // After consuming the frame, it calls connection.windowUpdater::processed, + // which decrements the count of unprocessed bytes, and possibly + // sends a window update to the peer. + // + // This method is called when connection.windowUpdater::processed + // will not be called, which can happen when consuming the frame + // fails, or when an empty DataFrame terminates the stream, + // or when the stream is cancelled while data is still + // sitting in its inputQ. In the later case, it is called for + // each frame that is dropped from the queue. + final void releaseUnconsumed(DataFrame df) { + windowUpdater.released(df.payloadLength()); + dropDataFrame(df); + } + + // This method can be called directly when a DataFrame is dropped + // before/without having been added to any Stream::inputQ. + // In that case, the number of unprocessed bytes hasn't been incremented + // by the stream, and does not need to be decremented. + // Otherwise, if the frame is dropped after having been added to the + // inputQ, releaseUnconsumed above should be called. final void dropDataFrame(DataFrame df) { if (isMarked(closedState, SHUTDOWN_REQUESTED)) return; if (debug.on()) { @@ -1416,11 +1444,12 @@ private void sendConnectionPreface() throws IOException { // Note that the default initial window size, not to be confused // with the initial window size, is defined by RFC 7540 as // 64K -1. - final int len = windowUpdater.initialWindowSize - DEFAULT_INITIAL_WINDOW_SIZE; - if (len != 0) { + final int len = windowUpdater.initialWindowSize - INITIAL_CONNECTION_WINDOW_SIZE; + assert len >= 0; + if (len > 0) { if (Log.channel()) { Log.logChannel("Sending initial connection window update frame: {0} ({1} - {2})", - len, windowUpdater.initialWindowSize, DEFAULT_INITIAL_WINDOW_SIZE); + len, windowUpdater.initialWindowSize, INITIAL_CONNECTION_WINDOW_SIZE); } windowUpdater.sendWindowUpdate(len); } @@ -1874,6 +1903,19 @@ public ConnectionWindowUpdateSender(Http2Connection connection, int getStreamId() { return 0; } + + @Override + protected boolean windowSizeExceeded(long received) { + if (connection.isOpen()) { + try { + connection.protocolError(ErrorFrame.FLOW_CONTROL_ERROR, + "connection window exceeded"); + } catch (IOException io) { + connection.shutdown(io); + } + } + return true; + } } /** diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java b/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java index 4c97fc9950d..221abc427f3 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/Stream.java @@ -160,14 +160,13 @@ class Stream extends ExchangeImpl { // send lock: prevent sending DataFrames after reset occurred. private final Lock sendLock = new ReentrantLock(); private final Lock stateLock = new ReentrantLock(); - /** * A reference to this Stream's connection Send Window controller. The * stream MUST acquire the appropriate amount of Send Window before * sending any data. Will be null for PushStreams, as they cannot send data. */ private final WindowController windowController; - private final WindowUpdateSender windowUpdater; + private final WindowUpdateSender streamWindowUpdater; @Override HttpConnection connection() { @@ -206,7 +205,8 @@ private void schedule() { int size = Utils.remaining(dsts, Integer.MAX_VALUE); if (size == 0 && finished) { inputQ.remove(); - connection.ensureWindowUpdated(df); // must update connection window + // consumed will not be called + connection.releaseUnconsumed(df); // must update connection window Log.logTrace("responseSubscriber.onComplete"); if (debug.on()) debug.log("incoming: onComplete"); sched.stop(); @@ -222,7 +222,11 @@ private void schedule() { try { subscriber.onNext(dsts); } catch (Throwable t) { - connection.dropDataFrame(df); // must update connection window + // Data frames that have been added to the inputQ + // must be released using releaseUnconsumed() to + // account for the amount of unprocessed bytes + // tracked by the connection.windowUpdater. + connection.releaseUnconsumed(df); throw t; } if (consumed(df)) { @@ -274,8 +278,12 @@ private void schedule() { private void drainInputQueue() { Http2Frame frame; while ((frame = inputQ.poll()) != null) { - if (frame instanceof DataFrame) { - connection.dropDataFrame((DataFrame)frame); + if (frame instanceof DataFrame df) { + // Data frames that have been added to the inputQ + // must be released using releaseUnconsumed() to + // account for the amount of unprocessed bytes + // tracked by the connection.windowUpdater. + connection.releaseUnconsumed(df); } } } @@ -301,12 +309,13 @@ private boolean consumed(DataFrame df) { boolean endStream = df.getFlag(DataFrame.END_STREAM); if (len == 0) return endStream; - connection.windowUpdater.update(len); - + connection.windowUpdater.processed(len); if (!endStream) { + streamWindowUpdater.processed(len); + } else { // Don't send window update on a stream which is // closed or half closed. - windowUpdater.update(len); + streamWindowUpdater.released(len); } // true: end of stream; false: more data coming @@ -376,8 +385,21 @@ public String toString() { } private void receiveDataFrame(DataFrame df) { - inputQ.add(df); - sched.runOrSchedule(); + try { + int len = df.payloadLength(); + if (len > 0) { + // we return from here if the connection is being closed. + if (!connection.windowUpdater.canBufferUnprocessedBytes(len)) return; + // we return from here if the stream is being closed. + if (closed || !streamWindowUpdater.canBufferUnprocessedBytes(len)) { + connection.releaseUnconsumed(df); + return; + } + } + inputQ.add(df); + } finally { + sched.runOrSchedule(); + } } /** Handles a RESET frame. RESET is always handled inline in the queue. */ @@ -461,7 +483,7 @@ CompletableFuture> sendBodyAsync() { this.responseHeadersBuilder = new HttpHeadersBuilder(); this.rspHeadersConsumer = new HeadersConsumer(); this.requestPseudoHeaders = createPseudoHeaders(request); - this.windowUpdater = new StreamWindowUpdateSender(connection); + this.streamWindowUpdater = new StreamWindowUpdateSender(connection); } private boolean checkRequestCancelled() { @@ -495,6 +517,8 @@ void incoming(Http2Frame frame) throws IOException { if (debug.on()) { debug.log("request cancelled or stream closed: dropping data frame"); } + // Data frames that have not been added to the inputQ + // can be released using dropDataFrame connection.dropDataFrame(df); } else { receiveDataFrame(df); @@ -1378,12 +1402,18 @@ void cancel(IOException cause) { @Override void onProtocolError(final IOException cause) { + onProtocolError(cause, ResetFrame.PROTOCOL_ERROR); + } + + void onProtocolError(final IOException cause, int code) { if (debug.on()) { - debug.log("cancelling exchange on stream %d due to protocol error: %s", streamid, cause.getMessage()); + debug.log("cancelling exchange on stream %d due to protocol error [%s]: %s", + streamid, ErrorFrame.stringForCode(code), + cause.getMessage()); } Log.logError("cancelling exchange on stream {0} due to protocol error: {1}\n", streamid, cause); // send a RESET frame and close the stream - cancelImpl(cause, ResetFrame.PROTOCOL_ERROR); + cancelImpl(cause, code); } void connectionClosing(Throwable cause) { @@ -1680,6 +1710,13 @@ String dbgString() { return dbgString = dbg; } } + + @Override + protected boolean windowSizeExceeded(long received) { + onProtocolError(new ProtocolException("stream %s flow control window exceeded" + .formatted(streamid)), ResetFrame.FLOW_CONTROL_ERROR); + return true; + } } /** diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/WindowUpdateSender.java b/src/java.net.http/share/classes/jdk/internal/net/http/WindowUpdateSender.java index 0bccc24e498..0affadddf15 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/WindowUpdateSender.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/WindowUpdateSender.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2023, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -31,16 +31,31 @@ import jdk.internal.net.http.frame.WindowUpdateFrame; import jdk.internal.net.http.common.Utils; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; +/** + * A class that tracks the amount of flow controlled + * data received on an HTTP/2 connection + */ abstract class WindowUpdateSender { final Logger debug = Utils.getDebugLogger(this::dbgString, Utils.DEBUG); + // The threshold at which window updates are sent in bytes final int limit; + // The flow control window in bytes + final int windowSize; final Http2Connection connection; - final AtomicInteger received = new AtomicInteger(); + // The amount of flow controlled data received and processed, in bytes, + // since the start of the window. + // The window is exhausted when received + unprocessed >= windowSize + final AtomicLong received = new AtomicLong(); + // The amount of flow controlled data received and unprocessed, in bytes, + // since the start of the window. + // The window is exhausted when received + unprocessed >= windowSize + final AtomicLong unprocessed = new AtomicLong(); final ReentrantLock sendLock = new ReentrantLock(); WindowUpdateSender(Http2Connection connection) { @@ -53,6 +68,7 @@ abstract class WindowUpdateSender { WindowUpdateSender(Http2Connection connection, int maxFrameSize, int initWindowSize) { this.connection = connection; + this.windowSize = initWindowSize; int v0 = Math.max(0, initWindowSize - maxFrameSize); int v1 = (initWindowSize + (maxFrameSize - 1)) / maxFrameSize; v1 = v1 * maxFrameSize / 2; @@ -66,16 +82,119 @@ abstract class WindowUpdateSender { maxFrameSize, initWindowSize, limit); } + // O for the connection window, > 0 for a stream window abstract int getStreamId(); + + /** + * {@return {@code true} if buffering the given amount of + * flow controlled data would not exceed the flow control + * window} + *

+ * This method is called before buffering and processing + * a DataFrame. The count of unprocessed bytes is incremented + * by the given amount, and checked against the number of + * available bytes in the flow control window. + *

+ * This method returns {@code true} if the bytes can be buffered + * without exceeding the flow control window, {@code false} + * if the flow control window is exceeded and corrective + * action (close/reset) has been taken. + *

+ * When this method returns true, either {@link #processed(int)} + * or {@link #released(int)} must eventually be called to release + * the bytes from the flow control window. + * + * @implSpec + * an HTTP/2 endpoint may disable its own flow control + * (see + * RFC 9113, section 5.2.1), in which case this + * method may return true even if the flow control window would + * be exceeded: that is, the flow control window is exceeded but + * the endpoint decided to take no corrective action. + * + * @param len a number of unprocessed bytes, which + * the caller wants to buffer. + */ + boolean canBufferUnprocessedBytes(int len) { + return !checkWindowSizeExceeded(unprocessed.addAndGet(len)); + } + + // adds the provided amount to the amount of already + // received and processed bytes and checks whether the + // flow control window is exceeded. If so, take + // corrective actions and return true. + private boolean checkWindowSizeExceeded(long len) { + // because windowSize is bound by Integer.MAX_VALUE + // we will never reach the point where received.get() + len + // could overflow + long rcv = received.get() + len; + return rcv > windowSize && windowSizeExceeded(rcv); + } + + /** + * Called after unprocessed buffered bytes have been + * processed, to release part of the flow control window + * + * @apiNote this method is called only when releasing bytes + * that where buffered after calling + * {@link #canBufferUnprocessedBytes(int)}. + * + * @param delta the amount of processed bytes to release + */ + void processed(int delta) { + long rest = unprocessed.addAndGet(-delta); + assert rest >= 0; + update(delta); + } + + /** + * Called when it is desired to release unprocessed bytes + * without processing them, or without triggering the + * sending of a window update. This method can be called + * instead of calling {@link #processed(int)}. + * When this method is called instead of calling {@link #processed(int)}, + * it should generally be followed by a call to {@link #update(int)}, + * unless the stream or connection is being closed. + * + * @apiNote this method should only be called to release bytes that + * have been buffered after calling {@link + * #canBufferUnprocessedBytes(int)}. + * + * @param delta the amount of bytes to release from the window + * + * @return the amount of remaining unprocessed bytes + */ + long released(int delta) { + long rest = unprocessed.addAndGet(-delta); + assert rest >= 0; + return rest; + } + + /** + * This method is called to update the flow control window, + * and possibly send a window update + * + * @apiNote this method can be called directly if a frame is + * dropped before calling {@link #canBufferUnprocessedBytes(int)}. + * Otherwise, either {@link #processed(int)} or {@link #released(int)} + * should be called, depending on whether sending a window update + * is desired or not. It is typically not desired to send an update + * if the stream or connection is being closed. + * + * @param delta the amount of bytes released from the window. + */ void update(int delta) { - int rcv = received.addAndGet(delta); + long rcv = received.addAndGet(delta); if (debug.on()) debug.log("update: %d, received: %d, limit: %d", delta, rcv, limit); + if (rcv > windowSize && windowSizeExceeded(rcv)) { + return; + } if (rcv > limit) { sendLock.lock(); try { - int tosend = received.get(); - if( tosend > limit) { + int tosend = (int)Math.min(received.get(), Integer.MAX_VALUE); + if (tosend > limit) { received.getAndAdd(-tosend); sendWindowUpdate(tosend); } @@ -87,6 +206,7 @@ void update(int delta) { void sendWindowUpdate(int delta) { if (debug.on()) debug.log("sending window update: %d", delta); + assert delta > 0 : "illegal window update delta: " + delta; connection.sendUnorderedFrame(new WindowUpdateFrame(getStreamId(), delta)); } @@ -104,4 +224,16 @@ String dbgString() { } } + /** + * Called when the flow control window size is exceeded + * This method may return false if flow control is disabled + * in this endpoint. + * + * @param received the amount of data received, which is greater + * than {@code windowSize} + * @return {@code true} if the error was reported to the peer + * and no further window update should be sent. + */ + protected abstract boolean windowSizeExceeded(long received); + } diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/frame/FramesDecoder.java b/src/java.net.http/share/classes/jdk/internal/net/http/frame/FramesDecoder.java index 72c7750eb43..7ebfa090830 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/frame/FramesDecoder.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/frame/FramesDecoder.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2018, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -463,6 +463,16 @@ private Http2Frame parseSettingsFrame(int frameLength, int streamid, int flags) int val = getInt(); if (id > 0 && id <= SettingsFrame.MAX_PARAM) { // a known parameter. Ignore otherwise + if (id == SettingsFrame.INITIAL_WINDOW_SIZE && val < 0) { + return new MalformedFrame(ErrorFrame.FLOW_CONTROL_ERROR, + "SettingsFrame with INITIAL_WINDOW_SIZE > 2^31 -1: " + + (val & 0xffffffffL)); + } + if (id == SettingsFrame.MAX_FRAME_SIZE && (val < 16384 || val > 16777215)) { + return new MalformedFrame(ErrorFrame.PROTOCOL_ERROR, + "SettingsFrame with MAX_FRAME_SIZE out of range: " + + (val & 0xffffffffL)); + } sf.setParameter(id, val); // TODO parameters validation } } @@ -530,7 +540,12 @@ private Http2Frame parseWindowUpdateFrame(int frameLength, int streamid, int fla return new MalformedFrame(ErrorFrame.FRAME_SIZE_ERROR, "WindowUpdateFrame length is "+ frameLength+", expected 4"); } - return new WindowUpdateFrame(streamid, getInt() & 0x7fffffff); + int update = getInt(); + if (update < 0) { + return new MalformedFrame(ErrorFrame.FLOW_CONTROL_ERROR, + "WindowUpdateFrame with value > 2^31 -1 " + (update & 0xffffffffL)); + } + return new WindowUpdateFrame(streamid, update & 0x7fffffff); } private Http2Frame parseContinuationFrame(int frameLength, int streamid, int flags) { diff --git a/src/java.net.http/share/classes/jdk/internal/net/http/frame/SettingsFrame.java b/src/java.net.http/share/classes/jdk/internal/net/http/frame/SettingsFrame.java index 8a011959417..b3b8164598f 100644 --- a/src/java.net.http/share/classes/jdk/internal/net/http/frame/SettingsFrame.java +++ b/src/java.net.http/share/classes/jdk/internal/net/http/frame/SettingsFrame.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2015, 2021, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2015, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -166,6 +166,11 @@ public synchronized void update(SettingsFrame updated) { // The initial value is 2^14 (16,384) octets. public static final int DEFAULT_MAX_FRAME_SIZE = 16 * K; + // Initial connection window size. This cannot be updated using the + // SETTINGS frame. + public static final int INITIAL_CONNECTION_WINDOW_SIZE = DEFAULT_INITIAL_WINDOW_SIZE; + + public static SettingsFrame defaultRFCSettings() { SettingsFrame f = new SettingsFrame(); f.setParameter(ENABLE_PUSH, DEFAULT_ENABLE_PUSH); diff --git a/src/java.net.http/share/classes/module-info.java b/src/java.net.http/share/classes/module-info.java index 5303e818866..c95d80657b8 100644 --- a/src/java.net.http/share/classes/module-info.java +++ b/src/java.net.http/share/classes/module-info.java @@ -57,9 +57,11 @@ * means that the cache is unbounded. * *

  • {@systemProperty jdk.httpclient.connectionWindowSize} (default: 2^26)
    - * The HTTP/2 client connection window size in bytes. The maximum size is 2^31-1. This value - * cannot be smaller than the stream window size, which can be configured through the - * {@code jdk.httpclient.windowsize} system property. + * The HTTP/2 client connection window size in bytes. Valid values are in the range + * [2^16-1, 2^31-1]. If an invalid value is provided, the default value is used. + * The implementation guarantees that the actual value will be no smaller than the stream + * window size, which can be configured through the {@code jdk.httpclient.windowsize} + * system property. *

  • *
  • {@systemProperty jdk.httpclient.disableRetryConnect} (default: false)
    * Whether automatic retry of connection failures is disabled. If false, then retries are @@ -150,7 +152,8 @@ * or 16kB)
    The buffer size used by the web socket implementation for socket writes. *

  • *
  • {@systemProperty jdk.httpclient.windowsize} (default: 16777216 or 16 MB)
    - * The HTTP/2 client stream window size in bytes. + * The HTTP/2 client stream window size in bytes. Valid values are in the range [2^14, 2^31-1]. + * If an invalid value is provided, the default value is used. *

  • *
  • {@systemProperty jdk.httpclient.auth.retrylimit} (default: 3)
    * The number of attempts the Basic authentication filter will attempt to retry a failed diff --git a/test/jdk/java/net/httpclient/GZIPInputStreamTest.java b/test/jdk/java/net/httpclient/GZIPInputStreamTest.java index e2a382dfc5e..7db41cde5d6 100644 --- a/test/jdk/java/net/httpclient/GZIPInputStreamTest.java +++ b/test/jdk/java/net/httpclient/GZIPInputStreamTest.java @@ -26,7 +26,7 @@ * @bug 8217264 * @summary Tests that you can map an InputStream to a GZIPInputStream * @library /test/lib /test/jdk/java/net/httpclient/lib - * @build jdk.test.lib.net.SimpleSSLContext jdk.httpclient.test.lib.common.HttpServerAdapters + * @build jdk.test.lib.net.SimpleSSLContext jdk.httpclient.test.lib.common.HttpServerAdapters ReferenceTracker * @run testng/othervm GZIPInputStreamTest */ diff --git a/test/jdk/java/net/httpclient/ProxySelectorTest.java b/test/jdk/java/net/httpclient/ProxySelectorTest.java index fd8f85fa6d7..a5407801bcd 100644 --- a/test/jdk/java/net/httpclient/ProxySelectorTest.java +++ b/test/jdk/java/net/httpclient/ProxySelectorTest.java @@ -376,7 +376,7 @@ public void setup() throws Exception { public void teardown() throws Exception { client = null; Thread.sleep(100); - AssertionError fail = TRACKER.check(500); + AssertionError fail = TRACKER.check(1500); try { proxy.stop(); authproxy.stop(); diff --git a/test/jdk/java/net/httpclient/http2/ConnectionFlowControlTest.java b/test/jdk/java/net/httpclient/http2/ConnectionFlowControlTest.java new file mode 100644 index 00000000000..6b0b3727ee2 --- /dev/null +++ b/test/jdk/java/net/httpclient/http2/ConnectionFlowControlTest.java @@ -0,0 +1,365 @@ +/* + * Copyright (c) 2024, Oracle and/or its affiliates. All rights reserved. + * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. + * + * This code is free software; you can redistribute it and/or modify it + * under the terms of the GNU General Public License version 2 only, as + * published by the Free Software Foundation. + * + * This code is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License + * version 2 for more details (a copy is included in the LICENSE file that + * accompanied this code). + * + * You should have received a copy of the GNU General Public License version + * 2 along with this work; if not, write to the Free Software Foundation, + * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. + * + * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA + * or visit www.oracle.com if you need additional information or have any + * questions. + */ + +/* + * @test + * @bug 8342075 + * @summary checks connection flow control + * @library /test/lib /test/jdk/java/net/httpclient/lib + * @build jdk.httpclient.test.lib.http2.Http2TestServer jdk.test.lib.net.SimpleSSLContext + * @run testng/othervm -Djdk.internal.httpclient.debug=true + * -Djdk.httpclient.connectionWindowSize=65535 + * -Djdk.httpclient.windowsize=16384 + * ConnectionFlowControlTest + */ + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.ProtocolException; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpHeaders; +import java.net.http.HttpRequest; +import java.net.http.HttpRequest.BodyPublishers; +import java.net.http.HttpResponse; +import java.net.http.HttpResponse.BodyHandler; +import java.net.http.HttpResponse.BodyHandlers; +import java.net.http.HttpResponse.BodySubscriber; +import java.net.http.HttpResponse.ResponseInfo; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map.Entry; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Flow.Subscription; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; + +import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestServer; +import jdk.httpclient.test.lib.http2.BodyOutputStream; +import jdk.httpclient.test.lib.http2.Http2Handler; +import jdk.httpclient.test.lib.http2.Http2TestExchange; +import jdk.httpclient.test.lib.http2.Http2TestExchangeImpl; +import jdk.httpclient.test.lib.http2.Http2TestServer; +import jdk.httpclient.test.lib.http2.Http2TestServerConnection; +import jdk.internal.net.http.common.HttpHeadersBuilder; +import jdk.internal.net.http.frame.ContinuationFrame; +import jdk.internal.net.http.frame.HeaderFrame; +import jdk.internal.net.http.frame.HeadersFrame; +import jdk.internal.net.http.frame.Http2Frame; +import jdk.internal.net.http.frame.SettingsFrame; +import jdk.test.lib.Utils; +import jdk.test.lib.net.SimpleSSLContext; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import static java.util.List.of; +import static java.util.Map.entry; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +public class ConnectionFlowControlTest { + + SSLContext sslContext; + HttpTestServer http2TestServer; // HTTP/2 ( h2c ) + HttpTestServer https2TestServer; // HTTP/2 ( h2 ) + String http2URI; + String https2URI; + final AtomicInteger reqid = new AtomicInteger(); + + + @DataProvider(name = "variants") + public Object[][] variants() { + return new Object[][] { + { http2URI }, + { https2URI }, + }; + } + + @Test(dataProvider = "variants") + void test(String uri) throws Exception { + System.out.printf("%ntesting %s%n", uri); + ConcurrentHashMap> responseSent = new ConcurrentHashMap<>(); + ConcurrentHashMap> responses = new ConcurrentHashMap<>(); + FCHttp2TestExchange.setResponseSentCB((s) -> responseSent.get(s).complete(s)); + int connectionWindowSize = Math.max(Integer.getInteger( + "jdk.httpclient.connectionWindowSize", 65535), 65535); + int windowSize = Math.max(Integer.getInteger( + "jdk.httpclient.windowsize", 65535), 16384); + int max = connectionWindowSize / windowSize + 2; + System.out.printf("connection window: %s, stream window: %s, will make %s requests%n", + connectionWindowSize, windowSize, max); + + try (HttpClient client = HttpClient.newBuilder().sslContext(sslContext).build()) { + String label = null; + + Throwable t = null; + try { + String[] keys = new String[max]; + for (int i = 0; i < max; i++) { + String query = "reqId=" + reqid.incrementAndGet(); + keys[i] = query; + URI uriWithQuery = URI.create(uri + "?" + query); + CompletableFuture sent = new CompletableFuture<>(); + responseSent.put(query, sent); + HttpRequest request = HttpRequest.newBuilder(uriWithQuery) + .POST(BodyPublishers.ofString("Hello there!")) + .build(); + System.out.println("\nSending request:" + uriWithQuery); + final HttpClient cc = client; + var response = cc.send(request, BodyHandlers.ofInputStream()); + responses.put(query, response); + String ckey = response.headers().firstValue("X-Connection-Key").get(); + if (label == null) label = ckey; + try { + if (i < max - 1) { + // the connection window might be exceeded at i == max - 2, which + // means that the last request could go on a new connection. + assertEquals(ckey, label, "Unexpected key for " + query); + } + } catch (AssertionError ass) { + // since we won't pull all responses, the client + // will not exit unless we ask it to shutdown now. + client.shutdownNow(); + throw ass; + } + } + try { + Thread.sleep(1000); + } catch (InterruptedException ie) { + // ignore + } + CompletableFuture allsent = CompletableFuture.allOf(responseSent.values().stream() + .toArray(CompletableFuture[]::new)); + allsent.get(); + for (int i = 0; i < max; i++) { + try { + String query = keys[i]; + var response = responses.get(keys[i]); + String ckey = response.headers().firstValue("X-Connection-Key").get(); + if (label == null) label = ckey; + assertEquals(ckey, label, "Unexpected key for " + query); + int wait = uri.startsWith("https://") ? 500 : 250; + try (InputStream is = response.body()) { + Thread.sleep(Utils.adjustTimeout(wait)); + is.readAllBytes(); + } + System.out.printf("%s did not fail: %s%n", query, response.statusCode()); + } catch (AssertionError t1) { + // since we won't pull all responses, the client + // will not exit unless we ask it to shutdown now. + client.shutdownNow(); + throw t1; + } catch (Throwable t0) { + System.out.println("Got EXPECTED: " + t0); + if (t0 instanceof ExecutionException) { + t0 = t0.getCause(); + } + t = t0; + try { + assertDetailMessage(t0, i); + } catch (AssertionError e) { + // since we won't pull all responses, the client + // will not exit unless we ask it to shutdown now. + client.shutdownNow(); + throw e; + } + } + } + } catch (Throwable t0) { + System.out.println("Got EXPECTED: " + t0); + if (t0 instanceof ExecutionException) { + t0 = t0.getCause(); + } + t = t0; + } + if (t == null) { + // we could fail here if we haven't waited long enough + fail("Expected exception, got all responses, should sleep time be raised?"); + } else { + assertDetailMessage(t, max); + } + String query = "reqId=" + reqid.incrementAndGet(); + URI uriWithQuery = URI.create(uri + "?" + query); + CompletableFuture sent = new CompletableFuture<>(); + responseSent.put(query, sent); + HttpRequest request = HttpRequest.newBuilder(uriWithQuery) + .POST(BodyPublishers.ofString("Hello there!")) + .build(); + System.out.println("\nSending last request:" + uriWithQuery); + var response = client.send(request, BodyHandlers.ofString()); + if (label != null) { + String ckey = response.headers().firstValue("X-Connection-Key").get(); + assertNotEquals(ckey, label); + System.out.printf("last request %s sent on different connection as expected:" + + "\n\tlast: %s\n\tprevious: %s%n", query, ckey, label); + } + } + } + + // Assertions based on implementation specific detail messages. Keep in + // sync with implementation. + static void assertDetailMessage(Throwable throwable, int iterationIndex) { + try { + Throwable cause = throwable; + while (cause != null) { + if (cause instanceof ProtocolException) { + if (cause.getMessage().contains("connection window exceeded")) { + System.out.println("Found expected exception: " + cause); + return; + } + } + cause = cause.getCause(); + } + throw new AssertionError( + "ProtocolException(\"protocol error: connection window exceeded\") not found", + throwable); + } catch (AssertionError e) { + System.out.println("Exception does not match expectation: " + throwable); + throwable.printStackTrace(System.out); + throw e; + } + } + + @BeforeTest + public void setup() throws Exception { + sslContext = new SimpleSSLContext().get(); + if (sslContext == null) + throw new AssertionError("Unexpected null sslContext"); + + var http2TestServer = new Http2TestServer("localhost", false, 0); + http2TestServer.addHandler(new Http2TestHandler(), "/http2/"); + this.http2TestServer = HttpTestServer.of(http2TestServer); + http2URI = "http://" + this.http2TestServer.serverAuthority() + "/http2/x"; + + var https2TestServer = new Http2TestServer("localhost", true, sslContext); + https2TestServer.addHandler(new Http2TestHandler(), "/https2/"); + this.https2TestServer = HttpTestServer.of(https2TestServer); + https2URI = "https://" + this.https2TestServer.serverAuthority() + "/https2/x"; + + // Override the default exchange supplier with a custom one to enable + // particular test scenarios + http2TestServer.setExchangeSupplier(FCHttp2TestExchange::new); + https2TestServer.setExchangeSupplier(FCHttp2TestExchange::new); + + this.http2TestServer.start(); + this.https2TestServer.start(); + } + + @AfterTest + public void teardown() throws Exception { + http2TestServer.stop(); + https2TestServer.stop(); + } + + static class Http2TestHandler implements Http2Handler { + + @Override + public void handle(Http2TestExchange t) throws IOException { + String query = t.getRequestURI().getRawQuery(); + + try (InputStream is = t.getRequestBody(); + OutputStream os = t.getResponseBody()) { + + byte[] bytes = is.readAllBytes(); + System.out.println("Server " + t.getLocalAddress() + " received:\n" + + t.getRequestURI() + ": " + new String(bytes, StandardCharsets.UTF_8)); + t.getResponseHeaders().setHeader("X-Connection-Key", t.getConnectionKey()); + + if (bytes.length == 0) bytes = "no request body!".getBytes(StandardCharsets.UTF_8); + int window = Math.max(16384, Integer.getInteger("jdk.httpclient.windowsize", 2*16*1024)); + final int maxChunkSize; + if (t instanceof FCHttp2TestExchange fct) { + maxChunkSize = Math.min(window, fct.conn.getMaxFrameSize()); + } else { + maxChunkSize = Math.min(window, SettingsFrame.MAX_FRAME_SIZE); + } + byte[] resp = bytes.length < maxChunkSize + ? bytes + : Arrays.copyOfRange(bytes, 0, maxChunkSize); + int max = (window / resp.length); + // send in chunks + t.sendResponseHeaders(200, 0); + int sent = 0; + for (int i=0; i<=max; i++) { + int len = Math.min(resp.length, window - sent); + if (len <= 0) break; + if (os instanceof BodyOutputStream bos) { + try { + // we don't wait for the stream window, but we want + // to wait for the connection window + bos.waitForStreamWindow(len); + } catch (InterruptedException ie) { + // ignore and continue... + } + } + ((BodyOutputStream) os).writeUncontrolled(resp, 0, len); + sent += len; + } + if (sent != window) fail("should have sent %s, sent %s".formatted(window, sent)); + } + if (t instanceof FCHttp2TestExchange fct) { + fct.responseSent(query); + } else { + fail("Exchange is not %s but %s" + .formatted(FCHttp2TestExchange.class.getName(), t.getClass().getName())); + } + } + } + + // A custom Http2TestExchangeImpl that overrides sendResponseHeaders to + // allow headers to be sent with a number of CONTINUATION frames. + static class FCHttp2TestExchange extends Http2TestExchangeImpl { + static volatile Consumer responseSentCB; + static void setResponseSentCB(Consumer responseSentCB) { + FCHttp2TestExchange.responseSentCB = responseSentCB; + } + + final Http2TestServerConnection conn; + FCHttp2TestExchange(int streamid, String method, HttpHeaders reqheaders, + HttpHeadersBuilder rspheadersBuilder, URI uri, InputStream is, + SSLSession sslSession, BodyOutputStream os, + Http2TestServerConnection conn, boolean pushAllowed) { + super(streamid, method, reqheaders, rspheadersBuilder, uri, is, sslSession, os, conn, pushAllowed); + this.conn = conn; + } + public void responseSent(String query) { + System.out.println("Server: response sent for " + query); + responseSentCB.accept(query); + } + + } +} diff --git a/test/jdk/java/net/httpclient/http2/StreamFlowControlTest.java b/test/jdk/java/net/httpclient/http2/StreamFlowControlTest.java new file mode 100644 index 00000000000..36b727e3a22 --- /dev/null +++ b/test/jdk/java/net/httpclient/http2/StreamFlowControlTest.java @@ -0,0 +1,342 @@ +/* + * Copyright (c) 2024, Oracle and/or its affiliates. All rights reserved. + * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. + * + * This code is free software; you can redistribute it and/or modify it + * under the terms of the GNU General Public License version 2 only, as + * published by the Free Software Foundation. + * + * This code is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License + * version 2 for more details (a copy is included in the LICENSE file that + * accompanied this code). + * + * You should have received a copy of the GNU General Public License version + * 2 along with this work; if not, write to the Free Software Foundation, + * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. + * + * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA + * or visit www.oracle.com if you need additional information or have any + * questions. + */ + +/* + * @test + * @bug 8342075 + * @library /test/lib /test/jdk/java/net/httpclient/lib + * @build jdk.httpclient.test.lib.http2.Http2TestServer jdk.test.lib.net.SimpleSSLContext + * @run testng/othervm -Djdk.internal.httpclient.debug=true + * -Djdk.httpclient.connectionWindowSize=65535 + * -Djdk.httpclient.windowsize=16384 + * StreamFlowControlTest + */ + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.ProtocolException; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpHeaders; +import java.net.http.HttpRequest; +import java.net.http.HttpRequest.BodyPublishers; +import java.net.http.HttpResponse; +import java.net.http.HttpResponse.BodyHandlers; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSession; + +import jdk.httpclient.test.lib.common.HttpServerAdapters.HttpTestServer; +import jdk.httpclient.test.lib.http2.BodyOutputStream; +import jdk.httpclient.test.lib.http2.Http2Handler; +import jdk.httpclient.test.lib.http2.Http2TestExchange; +import jdk.httpclient.test.lib.http2.Http2TestExchangeImpl; +import jdk.httpclient.test.lib.http2.Http2TestServer; +import jdk.httpclient.test.lib.http2.Http2TestServerConnection; +import jdk.internal.net.http.common.HttpHeadersBuilder; +import jdk.internal.net.http.frame.SettingsFrame; +import jdk.test.lib.Utils; +import jdk.test.lib.net.SimpleSSLContext; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; + +public class StreamFlowControlTest { + + SSLContext sslContext; + HttpTestServer http2TestServer; // HTTP/2 ( h2c ) + HttpTestServer https2TestServer; // HTTP/2 ( h2 ) + String http2URI; + String https2URI; + final AtomicInteger reqid = new AtomicInteger(); + + + @DataProvider(name = "variants") + public Object[][] variants() { + return new Object[][] { + { http2URI, false }, + { https2URI, false }, + { http2URI, true }, + { https2URI, true }, + }; + } + + + @Test(dataProvider = "variants") + void test(String uri, + boolean sameClient) + throws Exception + { + System.out.printf("%ntesting test(%s, %s)%n", uri, sameClient); + ConcurrentHashMap> responseSent = new ConcurrentHashMap<>(); + FCHttp2TestExchange.setResponseSentCB((s) -> responseSent.get(s).complete(s)); + + HttpClient client = null; + try { + int max = sameClient ? 10 : 3; + String label = null; + for (int i = 0; i < max; i++) { + if (!sameClient || client == null) + client = HttpClient.newBuilder().sslContext(sslContext).build(); + + String query = "reqId=" + reqid.incrementAndGet(); + URI uriWithQuery = URI.create(uri + "?" + query); + CompletableFuture sent = new CompletableFuture<>(); + responseSent.put(query, sent); + HttpRequest request = HttpRequest.newBuilder(uriWithQuery) + .POST(BodyPublishers.ofString("Hello there!")) + .build(); + System.out.println("\nSending request:" + uriWithQuery); + final HttpClient cc = client; + try { + HttpResponse response = cc.send(request, BodyHandlers.ofInputStream()); + if (sameClient) { + String key = response.headers().firstValue("X-Connection-Key").get(); + if (label == null) label = key; + assertEquals(key, label, "Unexpected key for " + query); + } + sent.join(); + // we have to pull to get the exception, but slow enough + // so that DataFrames are buffered up to the point that + // the window is exceeded... + int wait = uri.startsWith("https://") ? 500 : 350; + try (InputStream is = response.body()) { + Thread.sleep(Utils.adjustTimeout(wait)); + is.readAllBytes(); + } + // we could fail here if we haven't waited long enough + fail("Expected exception, got :" + response + ", should sleep time be raised?"); + } catch (IOException ioe) { + System.out.println("Got EXPECTED: " + ioe); + assertDetailMessage(ioe, i); + } finally { + if (!sameClient && client != null) { + client.close(); + client = null; + } + } + } + } finally { + if (sameClient && client != null) client.close(); + } + + } + + @Test(dataProvider = "variants") + void testAsync(String uri, + boolean sameClient) + { + System.out.printf("%ntesting testAsync(%s, %s)%n", uri, sameClient); + ConcurrentHashMap> responseSent = new ConcurrentHashMap<>(); + FCHttp2TestExchange.setResponseSentCB((s) -> responseSent.get(s).complete(s)); + + HttpClient client = null; + try { + int max = sameClient ? 5 : 3; + String label = null; + for (int i = 0; i < max; i++) { + if (!sameClient || client == null) + client = HttpClient.newBuilder().sslContext(sslContext).build(); + + String query = "reqId=" + reqid.incrementAndGet(); + URI uriWithQuery = URI.create(uri + "?" + query); + CompletableFuture sent = new CompletableFuture<>(); + responseSent.put(query, sent); + HttpRequest request = HttpRequest.newBuilder(uriWithQuery) + .POST(BodyPublishers.ofString("Hello there!")) + .build(); + System.out.println("\nSending request:" + uriWithQuery); + final HttpClient cc = client; + + Throwable t = null; + try { + HttpResponse response = cc.sendAsync(request, BodyHandlers.ofInputStream()).get(); + if (sameClient) { + String key = response.headers().firstValue("X-Connection-Key").get(); + if (label == null) label = key; + assertEquals(key, label, "Unexpected key for " + query); + } + sent.join(); + int wait = uri.startsWith("https://") ? 600 : 300; + try (InputStream is = response.body()) { + Thread.sleep(Utils.adjustTimeout(wait)); + is.readAllBytes(); + } + // we could fail here if we haven't waited long enough + fail("Expected exception, got :" + response + ", should sleep time be raised?"); + } catch (Throwable t0) { + System.out.println("Got EXPECTED: " + t0); + if (t0 instanceof ExecutionException) { + t0 = t0.getCause(); + } + t = t0; + } finally { + if (!sameClient && client != null) { + client.close(); + client = null; + } + } + assertDetailMessage(t, i); + } + } finally { + if (sameClient && client != null) client.close(); + } + } + + // Assertions based on implementation specific detail messages. Keep in + // sync with implementation. + static void assertDetailMessage(Throwable throwable, int iterationIndex) { + try { + Throwable cause = throwable; + while (cause != null) { + if (cause instanceof ProtocolException) { + if (cause.getMessage().matches("stream [0-9]+ flow control window exceeded")) { + System.out.println("Found expected exception: " + cause); + return; + } + } + cause = cause.getCause(); + } + throw new AssertionError( + "ProtocolException(\"stream X flow control window exceeded\") not found", + throwable); + } catch (AssertionError e) { + System.out.println("Exception does not match expectation: " + throwable); + throwable.printStackTrace(System.out); + throw e; + } + } + + @BeforeTest + public void setup() throws Exception { + sslContext = new SimpleSSLContext().get(); + if (sslContext == null) + throw new AssertionError("Unexpected null sslContext"); + + var http2TestServer = new Http2TestServer("localhost", false, 0); + http2TestServer.addHandler(new Http2TestHandler(), "/http2/"); + this.http2TestServer = HttpTestServer.of(http2TestServer); + http2URI = "http://" + this.http2TestServer.serverAuthority() + "/http2/x"; + + var https2TestServer = new Http2TestServer("localhost", true, sslContext); + https2TestServer.addHandler(new Http2TestHandler(), "/https2/"); + this.https2TestServer = HttpTestServer.of(https2TestServer); + https2URI = "https://" + this.https2TestServer.serverAuthority() + "/https2/x"; + + // Override the default exchange supplier with a custom one to enable + // particular test scenarios + http2TestServer.setExchangeSupplier(FCHttp2TestExchange::new); + https2TestServer.setExchangeSupplier(FCHttp2TestExchange::new); + + this.http2TestServer.start(); + this.https2TestServer.start(); + } + + @AfterTest + public void teardown() throws Exception { + http2TestServer.stop(); + https2TestServer.stop(); + } + + static class Http2TestHandler implements Http2Handler { + + @Override + public void handle(Http2TestExchange t) throws IOException { + String query = t.getRequestURI().getRawQuery(); + + try (InputStream is = t.getRequestBody(); + OutputStream os = t.getResponseBody()) { + + byte[] bytes = is.readAllBytes(); + System.out.println("Server " + t.getLocalAddress() + " received:\n" + + t.getRequestURI() + ": " + new String(bytes, StandardCharsets.UTF_8)); + t.getResponseHeaders().setHeader("X-Connection-Key", t.getConnectionKey()); + + if (bytes.length == 0) bytes = "no request body!".getBytes(StandardCharsets.UTF_8); + int window = Integer.getInteger("jdk.httpclient.windowsize", 2 * 16 * 1024); + final int maxChunkSize; + if (t instanceof FCHttp2TestExchange fct) { + maxChunkSize = Math.min(window, fct.conn.getMaxFrameSize()); + } else { + maxChunkSize = Math.min(window, SettingsFrame.MAX_FRAME_SIZE); + } + byte[] resp = bytes.length <= maxChunkSize + ? bytes + : Arrays.copyOfRange(bytes, 0, maxChunkSize); + int max = (window / resp.length) + 2; + // send in chunks + t.sendResponseHeaders(200, 0); + for (int i = 0; i <= max; i++) { + if (t instanceof FCHttp2TestExchange fct) { + try { + // we don't wait for the stream window, but we want + // to wait for the connection window + fct.conn.obtainConnectionWindow(resp.length); + } catch (InterruptedException ie) { + // ignore and continue... + } + } + ((BodyOutputStream) os).writeUncontrolled(resp, 0, resp.length); + } + } + if (t instanceof FCHttp2TestExchange fct) { + fct.responseSent(query); + } else fail("Exchange is not %s but %s" + .formatted(FCHttp2TestExchange.class.getName(), t.getClass().getName())); + } + } + + // A custom Http2TestExchangeImpl that overrides sendResponseHeaders to + // allow headers to be sent with a number of CONTINUATION frames. + static class FCHttp2TestExchange extends Http2TestExchangeImpl { + static volatile Consumer responseSentCB; + static void setResponseSentCB(Consumer responseSentCB) { + FCHttp2TestExchange.responseSentCB = responseSentCB; + } + + final Http2TestServerConnection conn; + FCHttp2TestExchange(int streamid, String method, HttpHeaders reqheaders, + HttpHeadersBuilder rspheadersBuilder, URI uri, InputStream is, + SSLSession sslSession, BodyOutputStream os, + Http2TestServerConnection conn, boolean pushAllowed) { + super(streamid, method, reqheaders, rspheadersBuilder, uri, is, sslSession, os, conn, pushAllowed); + this.conn = conn; + } + public void responseSent(String query) { + System.out.println("Server: response sent for " + query); + responseSentCB.accept(query); + } + + } +} diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/BodyOutputStream.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/BodyOutputStream.java index c091b7ecf9b..db5778ec3d9 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/BodyOutputStream.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/BodyOutputStream.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2016, 2023, Oracle and/or its affiliates. All rights reserved. + * Copyright (c) 2016, 2024, Oracle and/or its affiliates. All rights reserved. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. * * This code is free software; you can redistribute it and/or modify it @@ -25,6 +25,7 @@ import java.io.*; import java.nio.ByteBuffer; +import java.util.Objects; import jdk.internal.net.http.frame.DataFrame; import jdk.internal.net.http.frame.ResetFrame; @@ -65,6 +66,10 @@ void waitForWindow(int demand) throws InterruptedException { // first wait for the connection window conn.obtainConnectionWindow(demand); // now wait for the stream window + waitForStreamWindow(demand); + } + + public void waitForStreamWindow(int demand) throws InterruptedException { synchronized (this) { while (demand > 0) { int n = Math.min(demand, window); @@ -83,6 +88,7 @@ public void goodToGo() { @Override public void write(byte[] buf, int offset, int len) throws IOException { + Objects.checkFromIndexSize(offset, len, buf.length); if (closed) { throw new IOException("closed"); } @@ -104,6 +110,34 @@ public void write(byte[] buf, int offset, int len) throws IOException { } } + /** + * This method pushes frames onto the stack without checking + * for flow control, allowing the sender to bypass flow + * control for testing purposes + * @param buf data to send + * @param offset offset at which the data starts + * @param len length of the data to send + * @throws IOException if an I/O error occurs + */ + public void writeUncontrolled(byte[] buf, int offset, int len) + throws IOException { + Objects.checkFromIndexSize(offset, len, buf.length); + if (closed) { + throw new IOException("closed"); + } + + if (!goodToGo) { + throw new IllegalStateException("sendResponseHeaders must be called first"); + } + int max = conn.getMaxFrameSize(); + while (len > 0) { + int n = len > max ? max : len; + send(buf, offset, n, 0); + offset += n; + len -= n; + } + } + private void send(byte[] buf, int offset, int len, int flags) throws IOException { ByteBuffer buffer = ByteBuffer.allocate(len); buffer.put(buf, offset, len); diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java index b087a037222..bcb9fba8d8b 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java @@ -1295,7 +1295,7 @@ void registerStreamWindowUpdater(int streamid, Consumer r) { * * @param amount */ - synchronized void obtainConnectionWindow(int amount) throws InterruptedException { + public synchronized void obtainConnectionWindow(int amount) throws InterruptedException { while (amount > 0) { int n = Math.min(amount, sendWindow); amount -= n; @@ -1305,9 +1305,13 @@ synchronized void obtainConnectionWindow(int amount) throws InterruptedException } } - synchronized void updateConnectionWindow(int amount) { - sendWindow += amount; - notifyAll(); + void updateConnectionWindow(int amount) { + System.out.printf("sendWindow (window=%s, amount=%s) is now: %s%n", + sendWindow, amount, sendWindow + amount); + synchronized (this) { + sendWindow += amount; + notifyAll(); + } } // simplified output headers class. really just a type safe container From ec9d0ca6272837d1c869d6fd7e72f0dde1f2b15c Mon Sep 17 00:00:00 2001 From: Alexey Bakhtin Date: Fri, 21 Feb 2025 13:46:07 -0800 Subject: [PATCH 9/9] Add Http2TestExchange::getConnectionKey() from JDK-8335181 --- .../jdk/httpclient/test/lib/http2/Http2TestExchange.java | 6 ++++++ .../httpclient/test/lib/http2/Http2TestExchangeImpl.java | 5 +++++ .../test/lib/http2/Http2TestServerConnection.java | 4 ++++ 3 files changed, 15 insertions(+) diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java index 1a8b5d92af5..d2f2d761ede 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchange.java @@ -84,4 +84,10 @@ default void sendFrames(List frames) throws IOException { * It may also complete exceptionally */ CompletableFuture sendPing(); + + /** + * {@return the identification of the connection on which this exchange is being + * processed} + */ + String getConnectionKey(); } diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java index 0abcd51af63..c867b8bf040 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestExchangeImpl.java @@ -227,6 +227,11 @@ public void serverPush(URI uri, HttpHeaders headers, InputStream content) { } } + @Override + public String getConnectionKey() { + return conn.connectionKey(); + } + private boolean isHeadRequest() { return HEAD.equalsIgnoreCase(getRequestMethod()); } diff --git a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java index bcb9fba8d8b..a255b93cb68 100644 --- a/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java +++ b/test/jdk/java/net/httpclient/lib/jdk/httpclient/test/lib/http2/Http2TestServerConnection.java @@ -632,6 +632,10 @@ void createPrimordialStream(Http1InitialRequest request) throws IOException { }); } + final String connectionKey() { + return this.server.getAddress() + "->" + this.socket.getRemoteSocketAddress(); + } + // all other streams created here @SuppressWarnings({"rawtypes","unchecked"}) void createStream(HeaderFrame frame) throws IOException {