File tomcat-9.0.36-CVE-2024-34750.patch of Package tomcat.36510

Index: apache-tomcat-9.0.36-src/java/org/apache/coyote/AbstractProcessor.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/coyote/AbstractProcessor.java
+++ apache-tomcat-9.0.36-src/java/org/apache/coyote/AbstractProcessor.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
 
 import javax.servlet.RequestDispatcher;
 
@@ -847,7 +848,9 @@ public abstract class AbstractProcessor
         SocketWrapperBase<?> socketWrapper = getSocketWrapper();
         Iterator<DispatchType> dispatches = getIteratorAndClearDispatches();
         if (socketWrapper != null) {
-            synchronized (socketWrapper) {
+            Lock lock = socketWrapper.getLock();
+            lock.lock();
+            try {
                 /*
                  * This method is called when non-blocking IO is initiated by defining
                  * a read and/or write listener in a non-container thread. It is called
@@ -870,6 +873,8 @@ public abstract class AbstractProcessor
                     DispatchType dispatchType = dispatches.next();
                     socketWrapper.processSocket(dispatchType.getSocketStatus(), false);
                 }
+            } finally {
+                lock.unlock();
             }
         }
     }
Index: apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/AbstractStream.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/coyote/http2/AbstractStream.java
+++ apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/AbstractStream.java
@@ -33,6 +33,7 @@ abstract class AbstractStream {
     private static final StringManager sm = StringManager.getManager(AbstractStream.class);
 
     private final Integer identifier;
+    private final String idAsString;
 
     private volatile AbstractStream parentStream = null;
     private final Set<Stream> childStreams = Collections.newSetFromMap(new ConcurrentHashMap<>());
@@ -41,6 +42,7 @@ abstract class AbstractStream {
 
     AbstractStream(Integer identifier) {
         this.identifier = identifier;
+        this.idAsString = identifier.toString();
     }
 
 
@@ -48,6 +50,12 @@ abstract class AbstractStream {
         return identifier;
     }
 
+    /**
+     * @return the stream identifier as a String
+     */
+    final String getIdAsString() {
+        return idAsString;
+    }
 
     final int getIdAsInt() {
         return identifier.intValue();
Index: apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/Http2AsyncUpgradeHandler.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/coyote/http2/Http2AsyncUpgradeHandler.java
+++ apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/Http2AsyncUpgradeHandler.java
@@ -43,6 +43,8 @@ public class Http2AsyncUpgradeHandler ex
     // Because of the compression used, headers need to be written to the
     // network in the same order they are generated.
     private final Object headerWriteLock = new Object();
+    // Ensures thread triggers the stream reset is the first to send a RST frame
+    private final Object sendResetLock = new Object();
     private Throwable error = null;
     private IOException applicationIOE = null;
 
@@ -120,7 +122,7 @@ public class Http2AsyncUpgradeHandler ex
 
 
     @Override
-    void sendStreamReset(StreamException se) throws IOException {
+    void sendStreamReset(StreamStateMachine state, StreamException se) throws IOException {
         if (log.isDebugEnabled()) {
             log.debug(sm.getString("upgradeHandler.rst.debug", connectionId,
                     Integer.toString(se.getStreamId()), se.getError(), se.getMessage()));
@@ -136,9 +138,27 @@ public class Http2AsyncUpgradeHandler ex
         ByteUtil.set31Bits(rstFrame, 5, se.getStreamId());
         // Payload
         ByteUtil.setFourBytes(rstFrame, 9, se.getError().getCode());
-        socketWrapper.write(BlockingMode.SEMI_BLOCK, protocol.getWriteTimeout(),
-                TimeUnit.MILLISECONDS, null, SocketWrapperBase.COMPLETE_WRITE, errorCompletion,
-                ByteBuffer.wrap(rstFrame));
+
+        // Need to update state atomically with the sending of the RST
+        // frcccccbhcruununghbuvffffhghbjkfkgerrvvuhhgeufame else other threads currently working with this stream
+        // may see the state change and send a RST frame before the RST
+        // frame triggered by this thread. If that happens the client
+        // may see out of order RST frames which may hard to follow if
+        // the client is unaware the RST frames may be received out of
+        // order.
+        synchronized (sendResetLock) {
+            if (state != null) {
+                boolean active = state.isActive();
+                state.sendReset();
+                if (active) {
+                    decrementActiveRemoteStreamCount(getStream(se.getStreamId()));
+                }
+            }
+
+            socketWrapper.write(BlockingMode.SEMI_BLOCK, protocol.getWriteTimeout(),
+                    TimeUnit.MILLISECONDS, null, SocketWrapperBase.COMPLETE_WRITE, errorCompletion,
+                    ByteBuffer.wrap(rstFrame));
+        }
         handleAsyncException();
     }
 
@@ -184,7 +204,7 @@ public class Http2AsyncUpgradeHandler ex
             }
         }
         if (endOfStream) {
-            stream.sentEndOfStream();
+            sentEndOfStream(stream);
         }
     }
 
@@ -208,10 +228,7 @@ public class Http2AsyncUpgradeHandler ex
         header[3] = FrameType.DATA.getIdByte();
         if (finished) {
             header[4] = FLAG_END_OF_STREAM;
-            stream.sentEndOfStream();
-            if (!stream.isActive()) {
-                setConnectionTimeoutForStreamCount(activeRemoteStreamCount.decrementAndGet());
-            }
+            sentEndOfStream(stream);
         }
         if (writeable) {
             ByteUtil.set31Bits(header, 5, stream.getIdAsInt());
@@ -307,10 +324,7 @@ public class Http2AsyncUpgradeHandler ex
             header[3] = FrameType.DATA.getIdByte();
             if (finished) {
                 header[4] = FLAG_END_OF_STREAM;
-                sendfile.stream.sentEndOfStream();
-                if (!sendfile.stream.isActive()) {
-                    setConnectionTimeoutForStreamCount(activeRemoteStreamCount.decrementAndGet());
-                }
+                sentEndOfStream(sendfile.stream);
             }
             if (writeable) {
                 ByteUtil.set31Bits(header, 5, sendfile.stream.getIdAsInt());
@@ -368,10 +382,7 @@ public class Http2AsyncUpgradeHandler ex
             header[3] = FrameType.DATA.getIdByte();
             if (finished) {
                 header[4] = FLAG_END_OF_STREAM;
-                sendfile.stream.sentEndOfStream();
-                if (!sendfile.stream.isActive()) {
-                    setConnectionTimeoutForStreamCount(activeRemoteStreamCount.decrementAndGet());
-                }
+                sentEndOfStream(sendfile.stream);
             }
             if (writeable) {
                 ByteUtil.set31Bits(header, 5, sendfile.stream.getIdAsInt());
Index: apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/Http2UpgradeHandler.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/coyote/http2/Http2UpgradeHandler.java
+++ apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/Http2UpgradeHandler.java
@@ -276,6 +276,11 @@ class Http2UpgradeHandler extends Abstra
     }
 
 
+    protected void decrementActiveRemoteStreamCount(Stream stream) {
+        setConnectionTimeoutForStreamCount(stream.decrementAndGetActiveRemoteStreamCount());
+    }
+
+
     void processStreamOnContainerThread(StreamProcessor streamProcessor, SocketEvent event) {
         StreamRunnable streamRunnable = new StreamRunnable(streamProcessor, event);
         if (streamConcurrency == null) {
@@ -317,10 +322,18 @@ class Http2UpgradeHandler extends Abstra
         SocketState result = SocketState.CLOSED;
 
         try {
-            pingManager.sendPing(false);
-
             switch(status) {
             case OPEN_READ:
+                socketWrapper.getLock().lock();
+                try {
+                    if (!socketWrapper.canWrite()) {
+                        // Only send a ping if there is no other data waiting to be sent.
+                        // Ping manager will ensure they aren't sent too frequently.
+                        pingManager.sendPing(false);
+                    }
+                } finally {
+                    socketWrapper.getLock().unlock();
+                }
                 try {
                     // There is data to read so use the read timeout while
                     // reading frames ...
@@ -337,7 +350,7 @@ class Http2UpgradeHandler extends Abstra
                             // continue reading frames
                             Stream stream = getStream(se.getStreamId(), false);
                             if (stream == null) {
-                                sendStreamReset(se);
+                                sendStreamReset(null, se);
                             } else {
                                 stream.close(se);
                             }
@@ -519,7 +532,7 @@ class Http2UpgradeHandler extends Abstra
     }
 
 
-    void sendStreamReset(StreamException se) throws IOException {
+    void sendStreamReset(StreamStateMachine state, StreamException se) throws IOException {
 
         if (log.isDebugEnabled()) {
             log.debug(sm.getString("upgradeHandler.rst.debug", connectionId,
@@ -538,9 +551,26 @@ class Http2UpgradeHandler extends Abstra
         // Payload
         ByteUtil.setFourBytes(rstFrame, 9, se.getError().getCode());
 
-        synchronized (socketWrapper) {
+        // Need to update state atomically with the sending of the RST
+        // frame else other threads currently working with this stream
+        // may see the state change and send a RST frame before the RST
+        // frame triggered by this thread. If that happens the client
+        // may see out of order RST frames which may hard to follow if
+        // the client is unaware the RST frames may be received out of
+        // order.
+        socketWrapper.getLock().lock();
+        try {
+            if (state != null) {
+                boolean active = state.isActive();
+                state.sendReset();
+                if (active) {
+                    decrementActiveRemoteStreamCount(getStream(se.getStreamId()));
+                }
+            }
             socketWrapper.write(true, rstFrame, 0, rstFrame.length);
             socketWrapper.flush(true);
+        } finally {
+            socketWrapper.getLock().unlock();
         }
     }
 
@@ -626,7 +656,8 @@ class Http2UpgradeHandler extends Abstra
         byte[] payloadLength = new byte[3];
         ByteUtil.setThreeBytes(payloadLength, 0, len);
 
-        synchronized (socketWrapper) {
+        socketWrapper.getLock().lock();
+        try {
             socketWrapper.write(true, payloadLength, 0, payloadLength.length);
             socketWrapper.write(true, GOAWAY, 0, GOAWAY.length);
             socketWrapper.write(true, fixedPayload, 0, 8);
@@ -634,18 +665,23 @@ class Http2UpgradeHandler extends Abstra
                 socketWrapper.write(true, debugMsg, 0, debugMsg.length);
             }
             socketWrapper.flush(true);
+        } finally {
+            socketWrapper.getLock().unlock();
         }
     }
 
     void writeHeaders(Stream stream, int pushedStreamId, MimeHeaders mimeHeaders,
             boolean endOfStream, int payloadSize) throws IOException {
         // This ensures the Stream processing thread has control of the socket.
-        synchronized (socketWrapper) {
+        socketWrapper.getLock().lock();
+        try {
             doWriteHeaders(stream, pushedStreamId, mimeHeaders, endOfStream, payloadSize);
+        } finally {
+            socketWrapper.getLock().unlock();
         }
         stream.sentHeaders();
         if (endOfStream) {
-            stream.sentEndOfStream();
+            sentEndOfStream(stream);
         }
     }
 
@@ -751,29 +787,35 @@ class Http2UpgradeHandler extends Abstra
         header[3] = FrameType.DATA.getIdByte();
         if (finished) {
             header[4] = FLAG_END_OF_STREAM;
-            stream.sentEndOfStream();
-            if (!stream.isActive()) {
-                setConnectionTimeoutForStreamCount(activeRemoteStreamCount.decrementAndGet());
-            }
+            sentEndOfStream(stream);
         }
         if (writeable) {
             ByteUtil.set31Bits(header, 5, stream.getIdAsInt());
-            synchronized (socketWrapper) {
-                try {
-                    socketWrapper.write(true, header, 0, header.length);
-                    int orgLimit = data.limit();
-                    data.limit(data.position() + len);
-                    socketWrapper.write(true, data);
-                    data.limit(orgLimit);
-                    socketWrapper.flush(true);
-                } catch (IOException ioe) {
-                    handleAppInitiatedIOException(ioe);
-                }
+            socketWrapper.getLock().lock();
+            try {
+                socketWrapper.write(true, header, 0, header.length);
+                int orgLimit = data.limit();
+                data.limit(data.position() + len);
+                socketWrapper.write(true, data);
+                data.limit(orgLimit);
+                socketWrapper.flush(true);
+            } catch (IOException ioe) {
+                handleAppInitiatedIOException(ioe);
+            } finally {
+                socketWrapper.getLock().unlock();
             }
         }
     }
 
 
+    protected void sentEndOfStream(Stream stream) {
+        stream.sentEndOfStream();
+        if (!stream.isActive()) {
+            decrementActiveRemoteStreamCount(stream);
+        }
+    }
+
+
     /*
      * Handles an I/O error on the socket underlying the HTTP/2 connection when
      * it is triggered by application code (usually reading the request or
@@ -799,7 +841,8 @@ class Http2UpgradeHandler extends Abstra
         if (!stream.canWrite()) {
             return;
         }
-        synchronized (socketWrapper) {
+        socketWrapper.getLock().lock();
+        try {
             // Build window update frame for stream 0
             byte[] frame = new byte[13];
             ByteUtil.setThreeBytes(frame, 0,  4);
@@ -818,15 +861,24 @@ class Http2UpgradeHandler extends Abstra
                     throw ioe;
                 }
             }
+        } finally {
+            socketWrapper.getLock().unlock();
         }
     }
 
 
     protected void processWrites() throws IOException {
-        synchronized (socketWrapper) {
+        socketWrapper.getLock().lock();
+        try {
             if (socketWrapper.flush(false)) {
                 socketWrapper.registerWriteInterest();
+            } else {
+                // Only send a ping if there is no other data waiting to be sent.
+                // Ping manager will ensure they aren't sent too frequently.
+                pingManager.sendPing(false);
             }
+        } finally {
+            socketWrapper.getLock().unlock();
         }
     }
 
@@ -1068,12 +1120,21 @@ class Http2UpgradeHandler extends Abstra
     }
 
 
-    private Stream getStream(int streamId, boolean unknownIsError) throws ConnectionException {
+    Stream getStream(int streamId) {
         Integer key = Integer.valueOf(streamId);
-        Stream result = streams.get(key);
+        AbstractStream result = streams.get(key);
+        if (result instanceof Stream) {
+            return (Stream) result;
+        }
+        return null;
+    }
+
+
+    private Stream getStream(int streamId, boolean unknownIsError) throws ConnectionException {
+        Stream result = getStream(streamId);
         if (result == null && unknownIsError) {
             // Stream has been closed and removed from the map
-            throw new ConnectionException(sm.getString("upgradeHandler.stream.closed", key),
+            throw new ConnectionException(sm.getString("upgradeHandler.stream.closed", Integer.toString(streamId)),
                     Http2Error.PROTOCOL_ERROR);
         }
         return result;
@@ -1273,7 +1334,7 @@ class Http2UpgradeHandler extends Abstra
         if (localSettings.getMaxConcurrentStreams() < activeRemoteStreamCount.incrementAndGet()) {
             // If there are too many open streams, simply ignore the push
             // request.
-            setConnectionTimeoutForStreamCount(activeRemoteStreamCount.decrementAndGet());
+            decrementActiveRemoteStreamCount(associatedStream);
             return;
         }
 
@@ -1282,10 +1343,13 @@ class Http2UpgradeHandler extends Abstra
         // Synchronized since PUSH_PROMISE frames have to be sent in order. Once
         // the stream has been created we need to ensure that the PUSH_PROMISE
         // is sent before the next stream is created for a PUSH_PROMISE.
-        synchronized (socketWrapper) {
+        socketWrapper.getLock().lock();
+        try {
             pushStream = createLocalStream(request);
             writeHeaders(associatedStream, pushStream.getIdAsInt(), request.getMimeHeaders(),
                     false, Constants.DEFAULT_HEADERS_FRAME_SIZE);
+        } finally {
+            socketWrapper.getLock().unlock();
         }
 
         pushStream.sentPushPromise();
@@ -1412,10 +1476,7 @@ class Http2UpgradeHandler extends Abstra
     public void receivedEndOfStream(int streamId) throws ConnectionException {
         Stream stream = getStream(streamId, connectionState.get().isNewStreamAllowed());
         if (stream != null) {
-            stream.receivedEndOfStream();
-            if (!stream.isActive()) {
-                setConnectionTimeoutForStreamCount(activeRemoteStreamCount.decrementAndGet());
-            }
+            decrementActiveRemoteStreamCount(stream);
         }
     }
 
@@ -1441,6 +1502,7 @@ class Http2UpgradeHandler extends Abstra
             Stream stream = getStream(streamId, false);
             if (stream == null) {
                 stream = createRemoteStream(streamId);
+                activeRemoteStreamCount.incrementAndGet();
             }
             if (streamId < maxActiveRemoteStreamId) {
                 throw new ConnectionException(sm.getString("upgradeHandler.stream.old",
@@ -1523,9 +1585,8 @@ class Http2UpgradeHandler extends Abstra
             setMaxProcessedStream(streamId);
             if (stream.isActive()) {
                 if (stream.receivedEndOfHeaders()) {
-
-                    if (localSettings.getMaxConcurrentStreams() < activeRemoteStreamCount.incrementAndGet()) {
-                        setConnectionTimeoutForStreamCount(activeRemoteStreamCount.decrementAndGet());
+                    if (localSettings.getMaxConcurrentStreams() < activeRemoteStreamCount.get()) {
+                        decrementActiveRemoteStreamCount(stream);
                         // Ignoring maxConcurrentStreams increases the overhead count
                         increaseOverheadCount();
                         throw new StreamException(sm.getString("upgradeHandler.tooManyRemoteStreams",
@@ -1552,8 +1613,12 @@ class Http2UpgradeHandler extends Abstra
     @Override
     public void reset(int streamId, long errorCode) throws Http2Exception  {
         Stream stream = getStream(streamId, true);
+        boolean active = stream.isActive();
         stream.checkState(FrameType.RST);
         stream.receiveReset(errorCode);
+        if (active) {
+            decrementActiveRemoteStreamCount(stream);
+        }
     }
 
 
@@ -1598,9 +1663,12 @@ class Http2UpgradeHandler extends Abstra
                         "upgradeHandler.unexpectedAck", connectionId, getIdentifier()));
             }
         } else {
-            synchronized (socketWrapper) {
+            socketWrapper.getLock().lock();
+            try {
                 socketWrapper.write(true, SETTINGS_ACK, 0, SETTINGS_ACK.length);
                 socketWrapper.flush(true);
+            } finally {
+                socketWrapper.getLock().unlock();
             }
         }
     }
@@ -1704,7 +1772,8 @@ class Http2UpgradeHandler extends Abstra
             if (force || now - lastPingNanoTime > pingIntervalNano) {
                 lastPingNanoTime = now;
                 byte[] payload = new byte[8];
-                synchronized (socketWrapper) {
+                socketWrapper.getLock().lock();
+                try {
                     int sentSequence = ++sequence;
                     PingRecord pingRecord = new PingRecord(sentSequence, now);
                     inflightPings.add(pingRecord);
@@ -1712,6 +1781,8 @@ class Http2UpgradeHandler extends Abstra
                     socketWrapper.write(true, PING, 0, PING.length);
                     socketWrapper.write(true, payload, 0, payload.length);
                     socketWrapper.flush(true);
+                } finally {
+                    socketWrapper.getLock().lock();
                 }
             }
         }
@@ -1742,10 +1813,13 @@ class Http2UpgradeHandler extends Abstra
 
             } else {
                 // Client originated ping. Echo it back.
-                synchronized (socketWrapper) {
+                socketWrapper.getLock().lock();
+                try {
                     socketWrapper.write(true, PING_ACK, 0, PING_ACK.length);
                     socketWrapper.write(true, payload, 0, payload.length);
                     socketWrapper.flush(true);
+                } finally {
+                    socketWrapper.getLock().lock();
                 }
             }
         }
Index: apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/Stream.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/coyote/http2/Stream.java
+++ apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/Stream.java
@@ -26,6 +26,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.Locale;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Supplier;
 
 import org.apache.coyote.ActionCode;
@@ -85,7 +86,7 @@ class Stream extends AbstractStream impl
     private final StreamOutputBuffer streamOutputBuffer = new StreamOutputBuffer();
     private final Http2OutputBuffer http2OutputBuffer =
             new Http2OutputBuffer(coyoteResponse, streamOutputBuffer);
-
+    private final AtomicBoolean removedFromActiveCount = new AtomicBoolean(false);
 
     Stream(Integer identifier, Http2UpgradeHandler handler) {
         this(identifier, handler, null);
@@ -715,7 +716,7 @@ class Stream extends AbstractStream impl
                             se.getError()));
                 }
                 state.sendReset();
-                handler.sendStreamReset(se);
+                handler.sendStreamReset(state, se);
             } catch (IOException ioe) {
                 ConnectionException ce = new ConnectionException(
                         sm.getString("stream.reset.fail"), Http2Error.PROTOCOL_ERROR);
@@ -818,6 +819,18 @@ class Stream extends AbstractStream impl
         }
     }
 
+    int decrementAndGetActiveRemoteStreamCount() {
+        /*
+         * Protect against mis-counting of active streams. This method should only be called once per stream but since
+         * the count of active streams is used to enforce the maximum concurrent streams limit, make sure each stream is
+         * only removed from the active count exactly once.
+         */
+        if (removedFromActiveCount.compareAndSet(false, true)) {
+            return handler.activeRemoteStreamCount.decrementAndGet();
+        } else {
+            return handler.activeRemoteStreamCount.get();
+        }
+    }
 
     class StreamOutputBuffer implements HttpOutputBuffer, WriteBuffer.Sink {
 
Index: apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/StreamProcessor.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/coyote/http2/StreamProcessor.java
+++ apache-tomcat-9.0.36-src/java/org/apache/coyote/http2/StreamProcessor.java
@@ -75,7 +75,16 @@ class StreamProcessor extends AbstractPr
                         handler.getProtocol().getHttp11Protocol().addWaitingProcessor(this);
                     } else if (state == SocketState.CLOSED) {
                         handler.getProtocol().getHttp11Protocol().removeWaitingProcessor(this);
-                        if (!getErrorState().isConnectionIoAllowed()) {
+                        if (!stream.isInputFinished() && getErrorState().isIoAllowed()) {
+                            // The request has been processed but the request body has not been
+                            // fully read. This typically occurs when Tomcat rejects an upload
+                            // of some form (e.g. PUT or POST). Need to tell the client not to
+                            // send any more data on this stream (reset).
+                            StreamException se = new StreamException(
+                                    sm.getString("streamProcessor.cancel", stream.getConnectionId(),
+                                            stream.getIdAsString()), Http2Error.CANCEL, stream.getIdAsInt());
+                            stream.close(se);
+                        } else if (!getErrorState().isConnectionIoAllowed()) {
                             ConnectionException ce = new ConnectionException(sm.getString(
                                     "streamProcessor.error.connection", stream.getConnectionId(),
                                     stream.getIdentifier()), Http2Error.INTERNAL_ERROR);
@@ -386,13 +395,6 @@ class StreamProcessor extends AbstractPr
             setErrorState(ErrorState.CLOSE_NOW, e);
         }
 
-        if (!isAsync()) {
-            // If this is an async request then the request ends when it has
-            // been completed. The AsyncContext is responsible for calling
-            // endRequest() in that case.
-            endRequest();
-        }
-
         if (sendfileState == SendfileState.PENDING) {
             return SocketState.SENDFILE;
         } else if (getErrorState().isError()) {
@@ -435,30 +437,7 @@ class StreamProcessor extends AbstractPr
 
     @Override
     protected final SocketState dispatchEndRequest() throws IOException {
-        endRequest();
         return SocketState.CLOSED;
     }
 
-
-    private void endRequest() throws IOException {
-        if (!stream.isInputFinished() && getErrorState().isIoAllowed()) {
-            if (handler.hasAsyncIO() && !stream.isContentLengthInconsistent()) {
-                // Need an additional checks for asyncIO as the end of stream
-                // might have been set on the header frame but not processed
-                // yet. Checking for this here so the extra processing only
-                // occurs on the potential error condition rather than on every
-                // request.
-                return;
-            }
-            // The request has been processed but the request body has not been
-            // fully read. This typically occurs when Tomcat rejects an upload
-            // of some form (e.g. PUT or POST). Need to tell the client not to
-            // send any more data but only if a reset has not already been
-            // triggered.
-            StreamException se = new StreamException(
-                    sm.getString("streamProcessor.cancel", stream.getConnectionId(),
-                            stream.getIdentifier()), Http2Error.CANCEL, stream.getIdAsInt());
-            handler.sendStreamReset(se);
-        }
-    }
 }
Index: apache-tomcat-9.0.36-src/java/org/apache/tomcat/util/net/SocketProcessorBase.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/tomcat/util/net/SocketProcessorBase.java
+++ apache-tomcat-9.0.36-src/java/org/apache/tomcat/util/net/SocketProcessorBase.java
@@ -17,6 +17,7 @@
 package org.apache.tomcat.util.net;
 
 import java.util.Objects;
+import java.util.concurrent.locks.Lock;
 
 public abstract class SocketProcessorBase<S> implements Runnable {
 
@@ -37,7 +38,9 @@ public abstract class SocketProcessorBas
 
     @Override
     public final void run() {
-        synchronized (socketWrapper) {
+        Lock lock = socketWrapper.getLock();
+        lock.lock();
+        try {
             // It is possible that processing may be triggered for read and
             // write at the same time. The sync above makes sure that processing
             // does not occur in parallel. The test below ensures that if the
@@ -47,6 +50,8 @@ public abstract class SocketProcessorBas
                 return;
             }
             doRun();
+        } finally {
+            lock.unlock();
         }
     }
 
Index: apache-tomcat-9.0.36-src/java/org/apache/tomcat/util/net/SocketWrapperBase.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/java/org/apache/tomcat/util/net/SocketWrapperBase.java
+++ apache-tomcat-9.0.36-src/java/org/apache/tomcat/util/net/SocketWrapperBase.java
@@ -30,6 +30,8 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.juli.logging.Log;
 import org.apache.juli.logging.LogFactory;
@@ -44,6 +46,7 @@ public abstract class SocketWrapperBase<
 
     private E socket;
     private final AbstractEndpoint<E,?> endpoint;
+    private final Lock lock = new ReentrantLock();
 
     protected final AtomicBoolean closed = new AtomicBoolean(false);
 
@@ -136,6 +139,10 @@ public abstract class SocketWrapperBase<
         return endpoint;
     }
 
+    public Lock getLock() {
+        return lock;
+    }
+
     public Object getCurrentProcessor() {
         return currentProcessor.get();
     }
Index: apache-tomcat-9.0.36-src/test/org/apache/coyote/http2/Http2TestBase.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/test/org/apache/coyote/http2/Http2TestBase.java
+++ apache-tomcat-9.0.36-src/test/org/apache/coyote/http2/Http2TestBase.java
@@ -125,6 +125,11 @@ public abstract class Http2TestBase exte
 
 
     protected void validateHttp2InitialResponse() throws Exception {
+        validateHttp2InitialResponse(200);
+    }
+
+    protected void validateHttp2InitialResponse(long maxConcurrentStreams) throws Exception {
+
         // - 101 response acts as acknowledgement of the HTTP2-Settings header
         // Need to read 5 frames
         // - settings (server settings - must be first)
@@ -138,7 +143,7 @@ public abstract class Http2TestBase exte
         parser.readFrame(true);
         parser.readFrame(true);
 
-        Assert.assertEquals("0-Settings-[3]-[200]\n" +
+        Assert.assertEquals("0-Settings-[3]-[" + maxConcurrentStreams + "]\n" +
                 "0-Settings-End\n" +
                 "0-Settings-Ack\n" +
                 "0-Ping-[0,0,0,0,0,0,0,1]\n" +
@@ -558,15 +563,20 @@ public abstract class Http2TestBase exte
     }
 
     protected void enableHttp2(long maxConcurrentStreams, boolean tls) {
+        enableHttp2(maxConcurrentStreams, tls, 10000, 10000, 25000, 5000, 5000);
+    }
+
+    protected void enableHttp2(long maxConcurrentStreams, boolean tls, long readTimeout, long writeTimeout,
+                               long keepAliveTimeout, long streamReadTimout, long streamWriteTimeout) {
         Tomcat tomcat = getTomcatInstance();
         Connector connector = tomcat.getConnector();
         http2Protocol = new UpgradableHttp2Protocol();
         // Short timeouts for now. May need to increase these for CI systems.
-        http2Protocol.setReadTimeout(6000);
-        http2Protocol.setWriteTimeout(6000);
-        http2Protocol.setKeepAliveTimeout(15000);
-        http2Protocol.setStreamReadTimeout(3000);
-        http2Protocol.setStreamWriteTimeout(3000);
+        http2Protocol.setReadTimeout(readTimeout);
+        http2Protocol.setWriteTimeout(writeTimeout);
+        http2Protocol.setKeepAliveTimeout(keepAliveTimeout);
+        http2Protocol.setStreamReadTimeout(streamReadTimout);
+        http2Protocol.setStreamWriteTimeout(streamWriteTimeout);
         http2Protocol.setMaxConcurrentStreams(maxConcurrentStreams);
         connector.addUpgradeProtocol(http2Protocol);
         if (tls) {
Index: apache-tomcat-9.0.36-src/test/org/apache/coyote/http2/TestHttp2UpgradeHandler.java
===================================================================
--- apache-tomcat-9.0.36-src.orig/test/org/apache/coyote/http2/TestHttp2UpgradeHandler.java
+++ apache-tomcat-9.0.36-src/test/org/apache/coyote/http2/TestHttp2UpgradeHandler.java
@@ -69,4 +69,53 @@ public class TestHttp2UpgradeHandler ext
                 "3-EndOfStream\n", output.getTrace());
     }
 
+    @Test
+    public void testActiveConnectionCountAndClientTimeout() throws Exception {
+
+        enableHttp2(2, false, 10000, 10000, 4000, 2000, 2000);
+
+        Tomcat tomcat = getTomcatInstance();
+
+        Context ctxt = tomcat.addContext("", null);
+        Tomcat.addServlet(ctxt, "simple", new SimpleServlet());
+        ctxt.addServletMappingDecoded("/simple", "simple");
+
+        tomcat.start();
+
+        openClientConnection();
+        doHttpUpgrade();
+        sendClientPreface();
+        validateHttp2InitialResponse(2);
+
+        byte[] frameHeader = new byte[9];
+        ByteBuffer headersPayload = ByteBuffer.allocate(128);
+
+        byte[] dataFrameHeader = new byte[9];
+        ByteBuffer dataFramePayload = ByteBuffer.allocate(128);
+
+        // Should be able to make more than 2 requests even if they timeout
+        // since they should be removed from active connections once they
+        // timeout
+        for (int stream = 3; stream < 8; stream += 2) {
+            // Don't write the body. Allow the read to timeout.
+            buildPostRequest(frameHeader, headersPayload, false, dataFrameHeader, dataFramePayload, null, stream);
+            writeFrame(frameHeader, headersPayload);
+
+            // 500 response (triggered by IOException trying to read body that never arrived)
+            parser.readFrame();
+            Assert.assertTrue(output.getTrace(), output.getTrace().startsWith(
+                    stream + "-HeadersStart\n" +
+                            stream + "-Header-[:status]-[500]\n"));
+            output.clearTrace();
+
+            // reset frame
+            parser.readFrame();
+            Assert.assertEquals(stream + "-RST-[11]\n", output.getTrace());
+            output.clearTrace();
+
+            // Prepare buffers for re-use
+            headersPayload.clear();
+            dataFramePayload.clear();
+        }
+    }
 }
Index: apache-tomcat-9.0.36-src/webapps/docs/changelog.xml
===================================================================
--- apache-tomcat-9.0.36-src.orig/webapps/docs/changelog.xml
+++ apache-tomcat-9.0.36-src/webapps/docs/changelog.xml
@@ -120,6 +120,15 @@
         request triggers always error handling, regardless of whether the
         application swallows the exception. (markt)
       </fix>
+      <fix>
+        Improve the fix for RST frame ordering added in 10.1.0-M8 to avoid a
+        potential deadlock on some systems in non-default configurations.
+        (markt)
+      </fix>
+      <scode>
+        Simplify the closing on an HTTP/2 stream when an error condition is
+        present. (markt)
+      </scode>
     </changelog>
   </subsection>
   <subsection name="Coyote">
@@ -170,6 +179,27 @@
         by reporting problems at the end of the frame where the error was
         detected rather than at the end of the headers. (markt)
       </fix>
+      <fix>
+        Improve the fix for RST frame ordering added in 10.1.0-M8 to avoid a
+        potential deadlock on some systems in non-default configurations.
+        (markt)
+      </fix>
+      <fix>
+        <bug>66530</bug>: Correct a regression in the fix for bug
+        <bug>66442</bug> that meant that streams without a response body did not
+        decrement the active stream count when completing leading to
+        <code>ERR_HTTP2_SERVER_REFUSED_STREAM</code> for some connections.
+        (markt)
+      </fix>
+    </changelog>
+  </subsection>
+  <subsection name="Coyote">
+    <changelog>
+      <scode>
+        Refactor synchronization blocks locking on <code>SocketWrapper</code> to
+        use <code>ReentrantLock</code> to support users wishing to experiment
+        with project Loom. (markt)
+      </scode>
     </changelog>
   </subsection>
   <subsection name="Jasper">
@@ -210,6 +240,14 @@
       </fix>
     </changelog>
   </subsection>
+  <subsection name="Coyote">
+    <changelog>
+      <fix>
+        Do not send an HTTP/2 PING frame to measure round-trip time when it is
+        known that the HTTP/2 connection is not in a good state. (markt)
+      </fix>
+    </changelog>
+  </subsection>
   <subsection name="Other">
     <changelog>
       <update>
@@ -246,6 +284,10 @@
         Update the internal fork of Apache Commons FileUpload to 34eb241
         (2023-01-03, 2.0-SNAPSHOT). (markt)
       </update>
+      <fix>
+        Make counting of active HTTP/2 streams per connection more robust.
+        (markt)
+      </fix>
     </changelog>
   </subsection>
 </section>
@@ -1510,6 +1552,16 @@
       </fix>
     </changelog>
   </subsection>
+  <subsection name="Coyote">
+    <changelog>
+      <fix>
+        When an HTTP/2 stream was reset, the current active stream count was not
+        reduced. If enough resets occurred on a connection, the current active
+        stream count limit was reached and no new streams could be created on
+        that connection. (markt)
+      </fix>
+    </changelog>
+  </subsection>
   <subsection name="Other">
     <changelog>
       <scode>
openSUSE Build Service is sponsored by