Fixes retainability of special Chunks (#9073)

* Fixes #8993 - Retainability of special Chunks

* Restored Jetty 11's AsyncContentProducer and related classes in jetty-ee9-nested module (src and test).
* Introduced Retainable.canRetain().
* Removed Chunk.isTerminal() and replaced it with alternative method calls.
* Clarified AsyncContent.write() in case of Chunk.Error.
* Removed AsyncContent.write(Chunk, Callback) because it was making the API confusing.
  For example, AsyncContent.close() clashing with write(EOF, NOOP), or
  AsyncContent.fail(x) clashing with write(Chunk.Error, NOOP), etc.
* Improved usage of Chunk.from(..., Retainable).
* Improved usage of Chunk.slice().
* Using from() in MultiPart, rather than duplicating code.
* Fixed MultiPart.Parser.Listener.onPartContent() javadocs.
* Renamed non-retaining Chunk.from() to Chunk.asChunk().
* Removed Chunk.slice() methods, inlining them where necessary.
* Carefully reviewed all usages of read()-like methods that return a Retainable instance to make sure it is released.
* Updated HTTP/2 and HTTP/3 usages of Stream.Data to follow the correct retain/release semantic.

Signed-off-by: Simone Bordet <simone.bordet@gmail.com>
This commit is contained in:
Simone Bordet 2023-01-13 01:10:56 +01:00 committed by GitHub
parent de1f74b132
commit 46355c6110
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
88 changed files with 1440 additions and 1002 deletions

View File

@ -265,11 +265,7 @@ public class HTTPClientDocs
.onResponseBegin(response -> { /* ... */ })
.onResponseHeader((response, field) -> true)
.onResponseHeaders(response -> { /* ... */ })
.onResponseContentAsync((response, chunk, demander) ->
{
chunk.release();
demander.run();
})
.onResponseContentAsync((response, chunk, demander) -> demander.run())
.onResponseFailure((response, failure) -> { /* ... */ })
.onResponseSuccess(response -> { /* ... */ })
// Result hook.
@ -505,12 +501,12 @@ public class HTTPClientDocs
// nor demanded again until the demand callback is invoked.
return;
}
// Check if the chunk is the terminal one, in which case the
// Check if the chunk is last and empty, in which case the
// read/demand loop is done. Demanding again when the terminal
// chunk has been read will invoke the demand callback with
// the same terminal chunk, so this check must be present to
// avoid infinitely demanding and reading the terminal chunk.
if (chunk.isTerminal())
if (chunk.isLast() && !chunk.hasRemaining())
{
chunk.release();
return;

View File

@ -102,31 +102,27 @@ public class InputStreamResponseListener extends Listener.Adapter
{
if (LOG.isDebugEnabled())
LOG.debug("Skipped empty chunk {}", chunk);
chunk.release();
demander.run();
return;
}
boolean closed;
try (AutoLock.WithCondition l = lock.lock())
{
closed = this.closed;
if (!closed)
{
if (LOG.isDebugEnabled())
LOG.debug("Queueing chunk {}", chunk);
if (chunk.canRetain())
chunk.retain();
chunkCallbacks.add(new ChunkCallback(chunk, demander, response::abort));
l.signalAll();
return;
}
}
if (closed)
{
if (LOG.isDebugEnabled())
LOG.debug("InputStream closed, ignored chunk {}", chunk);
chunk.release();
response.abort(new AsynchronousCloseException());
}
if (LOG.isDebugEnabled())
LOG.debug("InputStream closed, ignored chunk {}", chunk);
response.abort(new AsynchronousCloseException());
}
@Override

View File

@ -155,12 +155,10 @@ public interface Response
try
{
onContent(response, chunk.getByteBuffer());
chunk.release();
demander.run();
}
catch (Throwable x)
{
chunk.release();
response.abort(x);
}
}
@ -190,10 +188,22 @@ public interface Response
response.abort(error.getCause());
return;
}
if (chunk.isTerminal())
if (chunk.isLast() && !chunk.hasRemaining())
{
chunk.release();
return;
}
onContent(response, chunk, () -> contentSource.demand(demandCallback));
try
{
onContent(response, chunk, () -> contentSource.demand(demandCallback));
chunk.release();
}
catch (Throwable x)
{
chunk.release();
response.abort(x);
}
}
}

View File

@ -574,7 +574,7 @@ public abstract class HttpReceiver
return _chunk;
// Retain the input chunk because its ByteBuffer will be referenced by the Inflater.
if (retain && _chunk.hasRemaining())
if (retain && _chunk.canRetain())
_chunk.retain();
if (LOG.isDebugEnabled())
LOG.debug("decoding: {}", _chunk);

View File

@ -308,6 +308,7 @@ public class ResponseNotifier
{
demultiplexerContentSource.onChunk(chunk);
}
chunk.release();
}
private void registerFailure(Throwable failure)
@ -368,6 +369,12 @@ public class ResponseNotifier
throw new UnsupportedOperationException();
}
@Override
public boolean canRetain()
{
throw new UnsupportedOperationException();
}
@Override
public void retain()
{
@ -383,7 +390,7 @@ public class ResponseNotifier
@Override
public String toString()
{
return "ALREADY_READ_CHUNK";
return "AlreadyReadChunk";
}
};
private final int index;
@ -401,9 +408,18 @@ public class ResponseNotifier
if (LOG.isDebugEnabled())
LOG.debug("Registering content in multiplexed content source #{} that contains {}", index, currentChunk);
if (currentChunk == null || currentChunk == ALREADY_READ_CHUNK)
this.chunk = chunk.slice();
{
if (chunk.hasRemaining())
chunk = Content.Chunk.asChunk(chunk.getByteBuffer().slice(), chunk.isLast(), chunk);
// Retain the slice because it is stored for later reads.
if (chunk.canRetain())
chunk.retain();
this.chunk = chunk;
}
else if (!currentChunk.isLast())
{
throw new IllegalStateException("Cannot overwrite chunk");
}
onDemandCallback();
}
@ -436,8 +452,8 @@ public class ResponseNotifier
}
Content.Chunk result = chunk;
if (result != null && !result.isTerminal())
chunk = ALREADY_READ_CHUNK;
if (result != null)
chunk = result.isLast() ? Content.Chunk.next(result) : ALREADY_READ_CHUNK;
if (LOG.isDebugEnabled())
LOG.debug("Content source #{} reading current chunk {}", index, result);
return result;

View File

@ -450,9 +450,10 @@ public class HttpReceiverOverHTTP extends HttpReceiver implements HttpParser.Res
if (chunk != null)
throw new IllegalStateException("Content generated with unconsumed content left");
RetainableByteBuffer networkBuffer = this.networkBuffer;
// Retain the chunk because it is stored for later use.
networkBuffer.retain();
chunk = Content.Chunk.from(buffer, false, networkBuffer);
chunk = Content.Chunk.asChunk(buffer, false, networkBuffer);
if (LOG.isDebugEnabled())
LOG.debug("Setting action to responseContentAvailable on {}", this);
if (getAndSetAction(this::responseContentAvailable) != null)

View File

@ -63,7 +63,6 @@ public class HttpClientAsyncContentTest extends AbstractHttpClientServerTest
.scheme(scenario.getScheme())
.onResponseContentAsync((response, chunk, demander) ->
{
chunk.release();
contentCount.incrementAndGet();
demanderRef.set(demander);
contentLatch.get().countDown();

View File

@ -115,7 +115,6 @@ public class HttpClientChunkedContentTest
client.newRequest("localhost", server.getLocalPort())
.onResponseContentAsync((response, chunk, demander) ->
{
chunk.release();
if (demanderRef.compareAndSet(null, demander))
firstContentLatch.countDown();
else

View File

@ -633,8 +633,7 @@ public class HttpClientTest extends AbstractHttpClientServerTest
AsyncRequestContent body = new AsyncRequestContent();
body.write(false, BufferUtil.allocate(512), Callback.NOOP);
body.write(false, BufferUtil.allocate(512), Callback.NOOP);
body.write(Content.Chunk.from(new IOException("explicitly_thrown_by_test")), Callback.NOOP);
body.write(false, BufferUtil.allocate(512), Callback.from(() -> body.fail(new IOException("explicitly_thrown_by_test"))));
CountDownLatch latch = new CountDownLatch(1);
client.newRequest("localhost", connector.getLocalPort())
.scheme(scenario.getScheme())
@ -1412,7 +1411,6 @@ public class HttpClientTest extends AbstractHttpClientServerTest
@Override
public void onContent(Response response, Content.Chunk chunk, Runnable demander)
{
chunk.release();
// Do not notify the callback yet.
demanderRef.set(demander);
contentLatch.countDown();

View File

@ -71,8 +71,7 @@ public class HttpClientUploadDuringServerShutdownTest
}
else
{
if (chunk.hasRemaining())
chunk.release();
chunk.release();
if (chunk.isLast())
break;
NanoTime.spinWait(TimeUnit.MICROSECONDS.toNanos(1));

View File

@ -86,6 +86,7 @@ public class AsyncRequestContentTest
Content.Chunk chunk = content.read();
assertNotNull(chunk);
chunk.release();
// Flush should return.
assertTrue(await(task, 5000));

View File

@ -484,11 +484,13 @@ public class HttpConnectionOverFCGI extends AbstractConnection implements IConne
HttpChannelOverFCGI channel = HttpConnectionOverFCGI.this.channel;
if (channel != null)
{
networkBuffer.retain();
Content.Chunk chunk = Content.Chunk.from(buffer, false, networkBuffer);
if (getAndSetAction(() -> channel.content(chunk)) != null)
throw new IllegalStateException();
return true;
// No need to call networkBuffer.retain() here, since we know
// that the action will be run before releasing the networkBuffer.
// The receiver of the chunk decides whether to consume/retain it.
Content.Chunk chunk = Content.Chunk.asChunk(buffer, false, networkBuffer);
if (getAndSetAction(() -> channel.content(chunk)) == null)
return true;
throw new IllegalStateException();
}
else
{

View File

@ -98,6 +98,9 @@ public class HttpReceiverOverFCGI extends HttpReceiver
{
if (this.chunk != null)
throw new IllegalStateException();
// Retain the chunk because it is stored for later reads.
if (chunk.canRetain())
chunk.retain();
this.chunk = chunk;
responseContentAvailable();
}

View File

@ -172,10 +172,6 @@ public class FastCGIProxyHandlerTest
{
response.abort(x);
}
finally
{
chunk.release();
}
})
.path(proxyContext.getContextPath() + path);
FutureResponseListener listener = new FutureResponseListener(request, length);

View File

@ -191,6 +191,9 @@ public class HttpStreamOverFCGI implements HttpStream
public void onContent(Content.Chunk chunk)
{
// Retain the chunk because it is stored for later reads.
if (chunk.canRetain())
chunk.retain();
_chunk = chunk;
}

View File

@ -371,8 +371,10 @@ public class ServerFCGIConnection extends AbstractConnection implements Connecti
LOG.debug("Request {} {} content {} on {}", request, streamType, buffer, stream);
if (stream != null)
{
networkBuffer.retain();
stream.onContent(Content.Chunk.from(buffer, false, networkBuffer));
// No need to call networkBuffer.retain() here.
// The receiver of the chunk decides whether to consume/retain it.
Content.Chunk chunk = Content.Chunk.asChunk(buffer, false, networkBuffer);
stream.onContent(chunk);
// Signal that the content is processed asynchronously, to ensure backpressure.
return true;
}

View File

@ -651,7 +651,6 @@ public class HttpClientTest extends AbstractHttpClientServerTest
.scheme(scheme)
.onResponseContentAsync((response, chunk, demander) ->
{
chunk.release();
contentCount.incrementAndGet();
demanderRef.set(demander);
contentLatch.get().countDown();

View File

@ -222,9 +222,9 @@ public class GZIPContentDecoder implements Destroyable
if (buffer.hasRemaining())
{
ByteBuffer chunk = buffer;
ByteBuffer decoded = buffer;
buffer = null;
if (decodedChunk(chunk))
if (decodedChunk(decoded))
return;
}
else if (_inflater.needsInput())

View File

@ -631,20 +631,13 @@ public class MultiPart
Content.Chunk chunk = part.getContent().read();
if (chunk == null || chunk instanceof Content.Chunk.Error)
yield chunk;
if (chunk.isLast())
{
if (!chunk.hasRemaining())
{
chunk.release();
chunk = Content.Chunk.EMPTY;
}
else
{
chunk = Content.Chunk.from(chunk.getByteBuffer(), false, chunk);
}
state = State.MIDDLE;
}
yield chunk;
if (!chunk.isLast())
yield chunk;
state = State.MIDDLE;
if (chunk.hasRemaining())
yield Content.Chunk.asChunk(chunk.getByteBuffer(), false, chunk);
chunk.release();
yield Content.Chunk.EMPTY;
}
case COMPLETE -> Content.Chunk.EOF;
};
@ -1190,11 +1183,15 @@ public class MultiPart
if (crContent)
{
crContent = false;
notifyPartContent(Content.Chunk.from(CR.slice(), false));
Content.Chunk partContentChunk = Content.Chunk.from(CR.slice(), false);
notifyPartContent(partContentChunk);
partContentChunk.release();
}
ByteBuffer content = ByteBuffer.wrap(boundaryFinder.getPattern(), 0, partialBoundaryMatch);
partialBoundaryMatch = 0;
notifyPartContent(Content.Chunk.from(content, false));
Content.Chunk partContentChunk = Content.Chunk.from(content, false);
notifyPartContent(partContentChunk);
partContentChunk.release();
return false;
}
}
@ -1209,7 +1206,7 @@ public class MultiPart
// if we found '\r\n--Boundary' then the '\r' is not content.
if (length > 0 && buffer.get(position + length - 1) == '\r')
--length;
Content.Chunk content = chunk.slice(position, length, true);
Content.Chunk content = asSlice(chunk, position, length, true);
buffer.position(position + boundaryOffset + boundaryFinder.getLength());
notifyPartContent(content);
notifyPartEnd();
@ -1231,7 +1228,7 @@ public class MultiPart
--sliceLimit;
}
int position = buffer.position();
Content.Chunk content = chunk.slice(position, sliceLimit - position, false);
Content.Chunk content = asSlice(chunk, position, sliceLimit - position, false);
buffer.position(limit);
if (content.hasRemaining())
notifyPartContent(content);
@ -1254,13 +1251,22 @@ public class MultiPart
--sliceLimit;
}
int position = buffer.position();
Content.Chunk content = chunk.slice(position, sliceLimit - position, false);
Content.Chunk content = asSlice(chunk, position, sliceLimit - position, false);
buffer.position(buffer.limit());
if (content.hasRemaining())
notifyPartContent(content);
return false;
}
private Content.Chunk asSlice(Content.Chunk chunk, int position, int length, boolean last)
{
if (chunk.isLast() && !chunk.hasRemaining())
return chunk;
if (length == 0)
return last ? Content.Chunk.EOF : Content.Chunk.EMPTY;
return Content.Chunk.asChunk(chunk.getByteBuffer().slice(position, length), last, chunk);
}
private void notifyPartBegin()
{
try
@ -1383,10 +1389,10 @@ public class MultiPart
/**
* <p>Callback method invoked when a part content {@code Chunk} has been parsed.</p>
* <p>The {@code Chunk} must be {@link Content.Chunk#release() released} when it
* has been consumed.</p>
* <p>The {@code Chunk} must be {@link Content.Chunk#retain()} retained} if it
* not consumed by this method (for example, stored away for later use).</p>
*
* @param chunk the part content chunk, must be released after use
* @param chunk the part content chunk
*/
default void onPartContent(Content.Chunk chunk)
{

View File

@ -283,6 +283,9 @@ public class MultiPartByteRanges extends CompletableFuture<MultiPartByteRanges.P
@Override
public void onPartContent(Content.Chunk chunk)
{
// Retain the chunk because it is stored for later use.
if (chunk.canRetain())
chunk.retain();
partChunks.add(chunk);
}

View File

@ -417,7 +417,6 @@ public class MultiPartFormData extends CompletableFuture<MultiPartFormData.Parts
if (maxFileSize >= 0 && fileSize > maxFileSize)
{
onFailure(new IllegalStateException("max file size exceeded: %d".formatted(maxFileSize)));
chunk.release();
return;
}
@ -438,13 +437,15 @@ public class MultiPartFormData extends CompletableFuture<MultiPartFormData.Parts
}
}
write(buffer);
chunk.release();
if (chunk.isLast())
close();
return;
}
}
}
// Retain the chunk because it is stored for later use.
if (chunk.canRetain())
chunk.retain();
partChunks.add(chunk);
}

View File

@ -44,6 +44,12 @@ public class Trailers implements Content.Chunk
return trailers;
}
@Override
public boolean canRetain()
{
return false;
}
@Override
public void retain()
{

View File

@ -658,7 +658,6 @@ public class MultiPartTest
public void onPartContent(Content.Chunk chunk)
{
events.offer("content last: %b length: %d".formatted(chunk.isLast(), chunk.getByteBuffer().remaining()));
chunk.release();
}
@Override
@ -690,6 +689,9 @@ public class MultiPartTest
@Override
public void onPartContent(Content.Chunk chunk)
{
// Retain the chunk because it is stored for later use.
if (chunk.canRetain())
chunk.retain();
partContent.add(chunk);
}

View File

@ -75,9 +75,11 @@ public class HttpReceiverOverHTTP2 extends HttpReceiver implements HTTP2Channel.
}
DataFrame frame = data.frame();
boolean last = frame.remaining() == 0 && frame.isEndStream();
if (last)
responseSuccess(getHttpExchange(), null);
return Content.Chunk.from(frame.getData(), last, data);
if (!last)
return Content.Chunk.asChunk(frame.getData(), last, data);
data.release();
responseSuccess(getHttpExchange(), null);
return Content.Chunk.EOF;
}
@Override

View File

@ -433,6 +433,12 @@ public interface Stream
super(new DataFrame(streamId, BufferUtil.EMPTY_BUFFER, true));
}
@Override
public boolean canRetain()
{
return false;
}
@Override
public void retain()
{

View File

@ -404,7 +404,6 @@ public class HTTP2Connection extends AbstractConnection implements WriteFlusher.
public void onData(DataFrame frame)
{
NetworkBuffer networkBuffer = producer.networkBuffer;
networkBuffer.retain();
session.onData(new StreamData(frame, networkBuffer));
}
@ -426,6 +425,12 @@ public class HTTP2Connection extends AbstractConnection implements WriteFlusher.
this.retainable = retainable;
}
@Override
public boolean canRetain()
{
return retainable.canRetain();
}
@Override
public void retain()
{
@ -463,6 +468,12 @@ public class HTTP2Connection extends AbstractConnection implements WriteFlusher.
return delegate.hasRemaining();
}
@Override
public boolean canRetain()
{
return delegate.canRetain();
}
@Override
public void retain()
{

View File

@ -248,7 +248,7 @@ public abstract class HTTP2Session extends ContainerLifeCycle implements Session
{
if (getRecvWindow() < 0)
{
onSessionFailure(ErrorCode.FLOW_CONTROL_ERROR.code, "session_window_exceeded", toCallback(data));
onSessionFailure(ErrorCode.FLOW_CONTROL_ERROR.code, "session_window_exceeded", Callback.NOOP);
}
else
{
@ -256,7 +256,7 @@ public abstract class HTTP2Session extends ContainerLifeCycle implements Session
{
// It's a bad client, it does not deserve to be
// treated gently by just resetting the stream.
onSessionFailure(ErrorCode.FLOW_CONTROL_ERROR.code, "stream_window_exceeded", toCallback(data));
onSessionFailure(ErrorCode.FLOW_CONTROL_ERROR.code, "stream_window_exceeded", Callback.NOOP);
}
else
{
@ -272,36 +272,12 @@ public abstract class HTTP2Session extends ContainerLifeCycle implements Session
// otherwise other requests will be stalled.
flowControl.onDataConsumed(this, null, flowControlLength);
if (isStreamClosed(streamId))
reset(null, new ResetFrame(streamId, ErrorCode.STREAM_CLOSED_ERROR.code), toCallback(data));
reset(null, new ResetFrame(streamId, ErrorCode.STREAM_CLOSED_ERROR.code), Callback.NOOP);
else
onSessionFailure(ErrorCode.PROTOCOL_ERROR.code, "unexpected_data_frame", toCallback(data));
onSessionFailure(ErrorCode.PROTOCOL_ERROR.code, "unexpected_data_frame", Callback.NOOP);
}
}
private Callback toCallback(Stream.Data data)
{
return new Callback()
{
@Override
public void succeeded()
{
data.release();
}
@Override
public void failed(Throwable x)
{
data.release();
}
@Override
public InvocationType getInvocationType()
{
return InvocationType.NON_BLOCKING;
}
};
}
private boolean isStreamClosed(int streamId)
{
return isLocalStream(streamId) ? isLocalStreamClosed(streamId) : isRemoteStreamClosed(streamId);
@ -2305,6 +2281,16 @@ public abstract class HTTP2Session extends ContainerLifeCycle implements Session
this.data = data;
this.stream = stream;
this.flowControlLength = flowControlLength;
// Since this class starts its own reference counter
// at 1, we need to retain the delegate Data object,
// so that the releases will be paired.
data.retain();
}
@Override
public boolean canRetain()
{
return data.canRetain();
}
@Override

View File

@ -414,7 +414,6 @@ public class HTTP2Stream implements Stream, Attachable, Closeable, Callback, Dum
{
if (LOG.isDebugEnabled())
LOG.debug("Data {} for already closed {}", data, this);
data.release();
reset(new ResetFrame(streamId, ErrorCode.STREAM_CLOSED_ERROR.code), Callback.NOOP);
return;
}
@ -424,7 +423,6 @@ public class HTTP2Stream implements Stream, Attachable, Closeable, Callback, Dum
// Just drop the frame.
if (LOG.isDebugEnabled())
LOG.debug("Data {} for already reset {}", data, this);
data.release();
return;
}
@ -436,7 +434,6 @@ public class HTTP2Stream implements Stream, Attachable, Closeable, Callback, Dum
{
if (LOG.isDebugEnabled())
LOG.debug("Invalid data length {} for {}", data, this);
data.release();
reset(new ResetFrame(streamId, ErrorCode.PROTOCOL_ERROR.code), Callback.NOOP);
return;
}
@ -448,6 +445,8 @@ public class HTTP2Stream implements Stream, Attachable, Closeable, Callback, Dum
private boolean offer(Data data)
{
// No need to retain the Data object because it
// has already been retained when it was created.
boolean process;
try (AutoLock ignored = lock.lock())
{

View File

@ -177,6 +177,9 @@ public abstract class HTTP2StreamEndPoint implements EndPoint
{
if (LOG.isDebugEnabled())
LOG.debug("closing {}, cause: {}", this, cause);
Stream.Data data = this.data.getAndSet(null);
if (data != null)
data.release();
shutdownOutput();
stream.close();
onClose(cause);

View File

@ -139,6 +139,11 @@ public class HttpStreamOverHTTP2 implements HttpStream, HTTP2Channel.Server
@Override
public Content.Chunk read()
{
// Tunnel requests do not have HTTP content, avoid
// returning chunks meant for a different protocol.
if (tunnelSupport != null)
return null;
while (true)
{
Content.Chunk chunk;
@ -154,8 +159,10 @@ public class HttpStreamOverHTTP2 implements HttpStream, HTTP2Channel.Server
if (data == null)
return null;
// The data instance should be released after readData() above;
// the chunk is stored below for later use, so should be retained;
// the two actions cancel each other, no need to further retain or release.
chunk = createChunk(data);
data.release();
// Some content is read, but the 100 Continue interim
// response has not been sent yet, then don't bother
@ -242,11 +249,11 @@ public class HttpStreamOverHTTP2 implements HttpStream, HTTP2Channel.Server
{
DataFrame frame = data.frame();
if (frame.isEndStream() && frame.remaining() == 0)
{
data.release();
return Content.Chunk.EOF;
// We need to retain because we are passing the ByteBuffer to the Chunk.
data.retain();
return Content.Chunk.from(frame.getData(), frame.isEndStream(), data);
}
return Content.Chunk.asChunk(frame.getData(), frame.isEndStream(), data);
}
@Override
@ -536,7 +543,7 @@ public class HttpStreamOverHTTP2 implements HttpStream, HTTP2Channel.Server
@Override
public Throwable consumeAvailable()
{
if (HttpMethod.CONNECT.is(_requestMetaData.getMethod()))
if (tunnelSupport != null)
return null;
return HttpStream.super.consumeAvailable();
}

View File

@ -571,6 +571,7 @@ public abstract class FlowControlStrategyTest
assertTrue(prepareLatch.await(5, TimeUnit.SECONDS));
// Second request will consume half of the remaining the session window.
List<Stream.Data> dataList2 = new ArrayList<>();
MetaData.Request request2 = newRequest("GET", HttpFields.EMPTY);
session.newStream(new HeadersFrame(request2, null, true), new Promise.Adapter<>(), new Stream.Listener()
{
@ -581,11 +582,13 @@ public abstract class FlowControlStrategyTest
if (!data.frame().isEndStream())
stream.demand();
// Do not release it to stall flow control.
dataList2.add(data);
}
});
// Third request will consume the whole session window, which is now stalled.
// A fourth request will not be able to receive data.
List<Stream.Data> dataList3 = new ArrayList<>();
MetaData.Request request3 = newRequest("GET", HttpFields.EMPTY);
session.newStream(new HeadersFrame(request3, null, true), new Promise.Adapter<>(), new Stream.Listener()
{
@ -596,6 +599,7 @@ public abstract class FlowControlStrategyTest
if (!data.frame().isEndStream())
stream.demand();
// Do not release it to stall flow control.
dataList3.add(data);
}
});
@ -624,6 +628,9 @@ public abstract class FlowControlStrategyTest
dataList1.forEach(Stream.Data::release);
assertTrue(latch.await(5, TimeUnit.SECONDS));
dataList2.forEach(Stream.Data::release);
dataList3.forEach(Stream.Data::release);
}
@Test
@ -754,6 +761,7 @@ public abstract class FlowControlStrategyTest
public void testClientExceedingSessionWindow() throws Exception
{
// On server, we don't consume the data.
List<Stream.Data> dataList = new ArrayList<>();
CountDownLatch serverCloseLatch = new CountDownLatch(1);
start(new ServerSessionListener()
{
@ -769,6 +777,7 @@ public abstract class FlowControlStrategyTest
Stream.Data data = stream.readData();
if (!data.frame().isEndStream())
stream.demand();
dataList.add(data);
}
};
}
@ -845,12 +854,15 @@ public abstract class FlowControlStrategyTest
assertTrue(clientGoAwayLatch.await(5, TimeUnit.SECONDS));
assertTrue(clientCloseLatch.await(5, TimeUnit.SECONDS));
assertTrue(serverCloseLatch.await(5, TimeUnit.SECONDS));
dataList.forEach(Stream.Data::release);
}
@Test
public void testClientExceedingStreamWindow() throws Exception
{
// On server, we don't consume the data.
List<Stream.Data> dataList = new ArrayList<>();
CountDownLatch serverCloseLatch = new CountDownLatch(1);
start(new ServerSessionListener()
{
@ -874,6 +886,7 @@ public abstract class FlowControlStrategyTest
Stream.Data data = stream.readData();
if (!data.frame().isEndStream())
stream.demand();
dataList.add(data);
}
};
}
@ -946,6 +959,8 @@ public abstract class FlowControlStrategyTest
assertTrue(clientGoAwayLatch.await(5, TimeUnit.SECONDS));
assertTrue(clientCloseLatch.await(5, TimeUnit.SECONDS));
assertTrue(serverCloseLatch.await(5, TimeUnit.SECONDS));
dataList.forEach(Stream.Data::release);
}
@Test

View File

@ -15,6 +15,8 @@ package org.eclipse.jetty.http2.tests;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
@ -387,6 +389,7 @@ public class GoAwayTest extends AbstractTest
{
int flowControlWindow = 32 * 1024;
List<Stream.Data> dataList = new ArrayList<>();
AtomicReference<Session> serverSessionRef = new AtomicReference<>();
CountDownLatch serverGoAwayLatch = new CountDownLatch(1);
CountDownLatch serverCloseLatch = new CountDownLatch(1);
@ -408,7 +411,8 @@ public class GoAwayTest extends AbstractTest
@Override
public void onDataAvailable(Stream stream)
{
stream.readData();
Stream.Data data = stream.readData();
dataList.add(data);
// Do not release the Data for this stream.
// Only send the response after reading the first DATA frame.
if (dataFrames.incrementAndGet() == 1)
@ -513,6 +517,8 @@ public class GoAwayTest extends AbstractTest
assertFalse(((HTTP2Session)serverSessionRef.get()).getEndPoint().isOpen());
assertFalse(((HTTP2Session)clientSession).getEndPoint().isOpen());
dataList.forEach(Stream.Data::release);
}
@Test

View File

@ -259,7 +259,7 @@ public class HttpClientTransportOverHTTP2Test extends AbstractTest
demander.run();
return;
}
if (!chunk.isTerminal())
if (chunk.hasRemaining())
contentCount.incrementAndGet();
chunk.release();
if (!chunk.isLast())

View File

@ -709,6 +709,7 @@ public class StreamResetTest extends AbstractTest
}
});
List<Stream.Data> dataList = new ArrayList<>();
AtomicLong received = new AtomicLong();
CountDownLatch latch = new CountDownLatch(1);
Session client = newClientSession(new Session.Listener() {});
@ -721,6 +722,7 @@ public class StreamResetTest extends AbstractTest
public void onDataAvailable(Stream stream)
{
Stream.Data data = stream.readData();
dataList.add(data);
// Do not release to stall the flow control window.
if (received.addAndGet(data.frame().getData().remaining()) == windowSize)
latch.countDown();
@ -744,6 +746,8 @@ public class StreamResetTest extends AbstractTest
HTTP2Session session = (HTTP2Session)sessions.iterator().next();
HTTP2Flusher flusher = session.getBean(HTTP2Flusher.class);
assertEquals(0, flusher.getFrameQueueSize());
dataList.forEach(Stream.Data::release);
}
@Test

View File

@ -127,6 +127,7 @@ public class TrailersTest extends AbstractTest
// No trailers yet.
assertThat(chunk, not(instanceOf(Trailers.class)));
chunk.release();
trailerLatch.countDown();
@ -143,6 +144,7 @@ public class TrailersTest extends AbstractTest
_request.demand(this::otherReads);
return;
}
chunk.release();
if (chunk instanceof Trailers contentTrailers)
{
HttpFields trailers = contentTrailers.getTrailers();

View File

@ -60,9 +60,11 @@ public class HttpReceiverOverHTTP3 extends HttpReceiver implements Stream.Client
}
ByteBuffer byteBuffer = data.getByteBuffer();
boolean last = !byteBuffer.hasRemaining() && data.isLast();
if (last)
responseSuccess(getHttpExchange(), null);
return Content.Chunk.from(byteBuffer, last, data);
if (!last)
return Content.Chunk.asChunk(byteBuffer, last, data);
data.release();
responseSuccess(getHttpExchange(), null);
return Content.Chunk.EOF;
}
@Override

View File

@ -412,6 +412,12 @@ public interface Stream
super(new DataFrame(BufferUtil.EMPTY_BUFFER, true));
}
@Override
public boolean canRetain()
{
return false;
}
@Override
public void retain()
{

View File

@ -334,6 +334,9 @@ public abstract class HTTP3Stream implements Stream, CyclicTimeouts.Expirable, A
public void onData(Data data)
{
// Retain the data because it is stored for later reads.
if (data.canRetain())
data.retain();
if (!dataRef.compareAndSet(null, data))
throw new IllegalStateException();

View File

@ -350,6 +350,12 @@ public abstract class HTTP3StreamConnection extends AbstractConnection
this.retainable = retainable;
}
@Override
public boolean canRetain()
{
return retainable.canRetain();
}
@Override
public void retain()
{
@ -432,7 +438,8 @@ public abstract class HTTP3StreamConnection extends AbstractConnection
}
else
{
networkBuffer.retain();
// No need to call networkBuffer.retain() here, since we know
// that the action will be run before releasing the networkBuffer.
data = new StreamData(frame, networkBuffer);
}

View File

@ -147,8 +147,10 @@ public class HttpStreamOverHTTP3 implements HttpStream
if (data == null)
return null;
// The data instance should be released after readData() above;
// the chunk is stored below for later use, so should be retained;
// the two actions cancel each other, no need to further retain or release.
chunk = createChunk(data);
data.release();
// Some content is read, but the 100 Continue interim
// response has not been sent yet, then don't bother
@ -204,8 +206,10 @@ public class HttpStreamOverHTTP3 implements HttpStream
return null;
}
// The data instance should be released after readData() above;
// the chunk is stored below for later use, so should be retained;
// the two actions cancel each other, no need to further retain or release.
Content.Chunk chunk = createChunk(data);
data.release();
try (AutoLock ignored = lock.lock())
{
@ -234,11 +238,11 @@ public class HttpStreamOverHTTP3 implements HttpStream
private Content.Chunk createChunk(Stream.Data data)
{
if (data == Stream.Data.EOF)
{
data.release();
return Content.Chunk.EOF;
// As we are passing the ByteBuffer to the Chunk we need to retain.
data.retain();
return Content.Chunk.from(data.getByteBuffer(), data.isLast(), data);
}
return Content.Chunk.asChunk(data.getByteBuffer(), data.isLast(), data);
}
@Override

View File

@ -80,10 +80,16 @@ public class DataDemandTest extends AbstractClientServerTest
{
// When resumed, demand all content until the last.
Stream.Data data = stream.readData();
if (data != null && data.isLast())
serverDataLatch.countDown();
else
stream.demand();
if (data != null)
{
data.release();
if (data.isLast())
{
serverDataLatch.countDown();
return;
}
}
stream.demand();
}
}
};
@ -129,7 +135,14 @@ public class DataDemandTest extends AbstractClientServerTest
if (serverStreamRef.compareAndSet(null, stream))
{
// Read only one chunk of data.
await().atMost(1, TimeUnit.SECONDS).until(() -> stream.readData() != null);
await().atMost(1, TimeUnit.SECONDS).until(() ->
{
Stream.Data data = stream.readData();
if (data == null)
return false;
data.release();
return true;
});
serverStreamLatch.countDown();
// Don't demand, just exit.
}
@ -137,10 +150,16 @@ public class DataDemandTest extends AbstractClientServerTest
{
// When resumed, demand all content until the last.
Stream.Data data = stream.readData();
if (data != null && data.isLast())
serverDataLatch.countDown();
else
stream.demand();
if (data != null)
{
data.release();
if (data.isLast())
{
serverDataLatch.countDown();
return;
}
}
stream.demand();
}
}
};
@ -198,6 +217,10 @@ public class DataDemandTest extends AbstractClientServerTest
serverStreamLatch.countDown();
break;
}
else
{
data.release();
}
}
// Do not demand after reading null data.
}
@ -205,10 +228,16 @@ public class DataDemandTest extends AbstractClientServerTest
{
// When resumed, demand all content until the last.
Stream.Data data = stream.readData();
if (data != null && data.isLast())
serverDataLatch.countDown();
else
stream.demand();
if (data != null)
{
data.release();
if (data.isLast())
{
serverDataLatch.countDown();
return;
}
}
stream.demand();
}
}
};
@ -293,6 +322,7 @@ public class DataDemandTest extends AbstractClientServerTest
}
if (dataRead.addAndGet(data.getByteBuffer().remaining()) == dataLength)
serverDataLatch.countDown();
data.release();
if (!data.isLast())
stream.demand();
}
@ -378,6 +408,7 @@ public class DataDemandTest extends AbstractClientServerTest
ByteBuffer buffer = ByteBuffer.wrap(bytesReceived);
datas.forEach(d -> buffer.put(d.getByteBuffer()));
assertArrayEquals(bytesSent, bytesReceived);
datas.forEach(Stream.Data::release);
}
@Test
@ -402,10 +433,16 @@ public class DataDemandTest extends AbstractClientServerTest
{
onDataAvailableCalls.incrementAndGet();
Stream.Data data = stream.readData();
if (data != null && data.isLast())
serverDataLatch.countDown();
else
stream.demand();
if (data != null)
{
data.release();
if (data.isLast())
{
serverDataLatch.countDown();
return;
}
}
stream.demand();
}
};
}
@ -531,6 +568,7 @@ public class DataDemandTest extends AbstractClientServerTest
if (data == null)
continue;
firstData = true;
data.release();
break;
}
@ -614,9 +652,8 @@ public class DataDemandTest extends AbstractClientServerTest
{
String content = StandardCharsets.UTF_8.decode(data.getByteBuffer()).toString();
assertEquals("hello", content);
data.release();
assertTrue(data.isLast());
data.release();
// Demand one more time, we should get an EOF.
stream.demand();
}
@ -625,6 +662,7 @@ public class DataDemandTest extends AbstractClientServerTest
assertNotNull(data);
assertTrue(data.isLast());
assertEquals(0, data.getByteBuffer().remaining());
data.release();
latch.countDown();
}
}

View File

@ -119,7 +119,7 @@ public class HttpClientTransportOverHTTP3Test extends AbstractClientServerTest
demander.run();
return;
}
if (!chunk.isTerminal())
if (chunk.hasRemaining())
contentCount.incrementAndGet();
chunk.release();
if (!chunk.isLast())

View File

@ -492,17 +492,22 @@ public class Content
}
/**
* <p>Creates a last/non-last Chunk with the given ByteBuffer, linked to the given Retainable.</p>
* <p>The {@link #retain()} and {@link #release()} methods of this Chunk will delegate to the
* given Retainable.</p>
* <p>Returns the given {@code ByteBuffer} and {@code last} arguments
* as a {@code Chunk}, linked to the given {@link Retainable}.</p>
* <p>The {@link #retain()} and {@link #release()} methods of this
* {@code Chunk} will delegate to the given {@code Retainable}.</p>
*
* @param byteBuffer the ByteBuffer with the bytes of this Chunk
* @param last whether the Chunk is the last one
* @param retainable the Retainable this Chunk links to
* @return a new Chunk
* @throws IllegalArgumentException if the {@code Retainable}
* {@link Retainable#canRetain() cannot be retained}
*/
static Chunk from(ByteBuffer byteBuffer, boolean last, Retainable retainable)
static Chunk asChunk(ByteBuffer byteBuffer, boolean last, Retainable retainable)
{
if (!retainable.canRetain())
throw new IllegalArgumentException("Cannot create chunk from non-retainable " + retainable);
if (byteBuffer.hasRemaining())
return new ByteBufferChunk.WithRetainable(byteBuffer, last, Objects.requireNonNull(retainable));
retainable.release();
@ -521,7 +526,7 @@ public class Content
}
/**
* <p>Returns the chunk that follows a chunk that has been consumed.</p>
* <p>Returns the chunk that follows the given chunk.</p>
* <table>
* <caption>Next Chunk</caption>
* <thead>
@ -569,63 +574,6 @@ public class Content
*/
boolean isLast();
/**
* <p>Returns a new {@code Chunk} whose {@code ByteBuffer} is a slice of the
* {@code ByteBuffer} of the source {@code Chunk} unless the source
* {@link #hasRemaining() has no remaining byte} in which case:</p>
* <ul>
* <li>{@code this} is returned if it is an instance of {@link Error}</li>
* <li>{@link #EOF} is returned if {@link #isLast()} is {@code true}</li>
* <li>{@link #EMPTY} is returned if {@link #isLast()} is {@code false}</li>
* </ul>
* <p>If the source has remaining bytes, the returned {@code Chunk} retains
* the source {@code Chunk} and it is linked to it via
* {@link #from(ByteBuffer, boolean, Retainable)}.</p>
*
* @return a new {@code Chunk} retained from the source {@code Chunk} with a slice
* of the source {@code Chunk}'s {@code ByteBuffer}
*/
default Chunk slice()
{
if (isTerminal())
return this;
if (!hasRemaining())
return EMPTY;
retain();
return from(getByteBuffer().slice(), isLast(), this);
}
/**
* <p>Returns a new {@code Chunk} whose {@code ByteBuffer} is a slice of the
* {@code ByteBuffer} of this {@code Chunk}.</p>
* <p>The returned {@code Chunk} is:</p>
* <ul>
* <li>{@code this}, if this {@code Chunk} is {@link #isTerminal() terminal}</li>
* <li>{@link #EMPTY}, if {@code length == 0 && !last}</li>
* <li>{@link #EOF}, if {@code length == 0 && last}</li>
* <li>a new {@code Chunk} whose {@code ByteBuffer} is a slice of the
* {@code ByteBuffer} of this {@code Chunk}, from the given {@code position}
* and for the given {@code length} bytes.</li>
* </ul>
* <p>The returned {@code Chunk} retains the source {@code Chunk} and it is linked
* to it via {@link #from(ByteBuffer, boolean, Retainable)}.</p>
*
* @param position the position at which the slice begins
* @param length the length of the slice
* @param last whether the new Chunk is last
* @return a new {@code Chunk} retained from the source {@code Chunk} with a slice
* of the source {@code Chunk}'s {@code ByteBuffer}
*/
default Chunk slice(int position, int length, boolean last)
{
if (isTerminal())
return this;
if (length == 0)
return last ? EOF : EMPTY;
retain();
return from(getByteBuffer().slice(position, length), last, this);
}
/**
* @return the number of bytes remaining in this Chunk
*/
@ -668,31 +616,14 @@ public class Content
*/
default int skip(int length)
{
if (length == 0)
return 0;
ByteBuffer byteBuffer = getByteBuffer();
length = Math.min(byteBuffer.remaining(), length);
byteBuffer.position(byteBuffer.position() + length);
return length;
}
/**
* <p>Returns whether this Chunk is a <em>terminal</em> chunk.</p>
* <p>A terminal chunk is a Chunk that {@link #isLast()} is true
* and has no remaining bytes.</p>
* <p><em>Terminal</em> chunks cannot be lifecycled using the
* {@link Retainable} contract like other chunks: they always throw
* {@link UnsupportedOperationException} on {@link #retain()} and
* always return {@code true} on {@link #release()}. As such, they
* cannot contain a recyclable buffer and calling their
* {@link #release()} method isn't necessary, although harmless.
* </p>
*
* @return whether this Chunk is a terminal chunk
*/
default boolean isTerminal()
{
return isLast() && !hasRemaining();
}
/**
* <p>A chunk that wraps a failure.</p>
* <p>Error Chunks are always last and have no bytes to read,
@ -726,6 +657,12 @@ public class Content
return true;
}
@Override
public boolean canRetain()
{
return false;
}
@Override
public void retain()
{

View File

@ -26,6 +26,18 @@ import java.util.concurrent.atomic.AtomicInteger;
*/
public interface Retainable
{
/**
* <p>Returns whether this resource can be retained, that is whether {@link #retain()}
* can be called safely.</p>
* <p>Implementations may decide that special resources are not retainable (for example,
* {@code static} constants) so calling {@link #retain()} is not safe because it may throw.</p>
* <p>Calling {@link #release()} on those special resources is typically allowed, and
* it is a no-operation.</p>
*
* @return whether it is safe to call {@link #retain()}
*/
boolean canRetain();
/**
* <p>Retains this resource, incrementing the reference count.</p>
*/
@ -40,6 +52,9 @@ public interface Retainable
*/
boolean release();
/**
* A wrapper of {@link Retainable} instances.
*/
class Wrapper implements Retainable
{
private final Retainable wrapped;
@ -54,6 +69,12 @@ public interface Retainable
return wrapped;
}
@Override
public boolean canRetain()
{
return getWrapped().canRetain();
}
@Override
public void retain()
{
@ -108,6 +129,12 @@ public interface Retainable
throw new IllegalStateException("acquired while in use " + this);
}
@Override
public boolean canRetain()
{
return true;
}
@Override
public void retain()
{

View File

@ -53,22 +53,13 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
* <ul>
* <li>immediately with a failure when this instance is closed or already in error</li>
* <li>successfully when the {@link Content.Chunk} returned by {@link #read()} is released</li>
* <li>successfully just before the {@link Content.Chunk} is returned if the latter {@link Content.Chunk#hasRemaining() has no remaining byte}</li>
* <li>successfully just before the {@link Content.Chunk} is returned by {@link #read()},
* if the chunk {@link Content.Chunk#canRetain() cannot be retained}</li>
* </ul>
*/
@Override
public void write(boolean last, ByteBuffer byteBuffer, Callback callback)
{
// Since the contract is that the callback has to be succeeded when
// the chunk returned by read() is released, and since it is not
// possible to create chunks with no remaining byte, when the byte
// buffer is empty we need to replace it with EOF / EMPTY and cannot
// be notified about the release of the latter two.
// This is why read() succeeds the callback if it has no remaining
// byte, meaning it is either EOF or EMPTY. The callback is succeeded
// once and only once, but that happens either during read() if the
// byte buffer is empty or during Chunk.release() if it contains at
// least one byte.
Content.Chunk chunk;
if (byteBuffer.hasRemaining())
chunk = Content.Chunk.from(byteBuffer, last, callback::succeeded);
@ -77,46 +68,6 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
offer(chunk, callback);
}
/**
* <p>Writes the given {@link Content.Chunk}, notifying the {@link Callback} when the
* write is complete.</p>
* <p>The callback completes:</p>
* <ul>
* <li>immediately with a failure when this instance is closed or already in error</li>
* <li>immediately with a failure when the written chunk is an instance of {@link Content.Chunk.Error}</li>
* <li>successfully when the {@link Content.Chunk} returned by {@link #read()} is released</li>
* <li>successfully just before the {@link Content.Chunk} is returned if the latter {@link Content.Chunk#hasRemaining() has no remaining byte}</li>
* </ul>
*
* @param chunk the Content.Chunk to write
* @param callback the callback to notify when the write operation is complete
*/
public void write(Content.Chunk chunk, Callback callback)
{
// Non-empty, chunks have to be wrapped to bind the succeeding
// of the callback to the release of the chunk. Empty chunks
// cannot be wrapped, so the callback is succeeded in read()
// for them.
Content.Chunk c;
if (chunk.isTerminal())
{
c = chunk;
}
else if (!chunk.hasRemaining())
{
c = Content.Chunk.EMPTY;
}
else
{
c = Content.Chunk.from(chunk.getByteBuffer(), chunk.isLast(), () ->
{
chunk.release();
callback.succeeded();
});
}
offer(c, callback);
}
/**
* The callback is stored to be failed in case fail() is called
* or succeeded if and only if the chunk is terminal, as non-terminal
@ -124,6 +75,12 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
*/
private void offer(Content.Chunk chunk, Callback callback)
{
if (chunk instanceof Content.Chunk.Error)
{
callback.failed(new IllegalArgumentException("Cannot not write Chunk.Error instances, call fail(Throwable) instead"));
return;
}
Throwable failure = null;
boolean wasEmpty = false;
try (AutoLock ignored = lock.lock())
@ -136,15 +93,11 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
{
failure = errorChunk.getCause();
}
else if (chunk instanceof Content.Chunk.Error error)
{
errorChunk = error;
failure = errorChunk.getCause();
wasEmpty = chunks.isEmpty();
}
else
{
wasEmpty = chunks.isEmpty();
// No need to retain the chunk, because it's created internally
// from a ByteBuffer and it will be released by the caller of read().
chunks.offer(new ChunkCallback(chunk, callback));
if (chunk.isLast())
{
@ -162,39 +115,36 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
public void flush() throws IOException
{
try (AutoLock.WithCondition l = lock.lock())
try (AutoLock.WithCondition condition = lock.lock())
{
try
while (true)
{
while (true)
// Always wrap the exception to make sure
// the stack trace comes from flush().
if (errorChunk != null)
throw new IOException(errorChunk.getCause());
if (chunks.isEmpty())
return;
// Special case for a last empty chunk that may not be read.
if (writeClosed && chunks.size() == 1)
{
// Always wrap the exception to make sure
// the stack trace comes from flush().
if (errorChunk != null)
throw new IOException(errorChunk.getCause());
if (chunks.isEmpty())
Content.Chunk chunk = chunks.peek().chunk();
if (chunk.isLast() && !chunk.hasRemaining())
return;
// Special case for a last empty chunk that may not be read.
if (writeClosed && chunks.size() == 1)
{
Content.Chunk chunk = chunks.peek().chunk();
if (chunk.isTerminal())
return;
}
l.await();
}
condition.await();
}
catch (InterruptedException x)
{
throw new InterruptedIOException();
}
}
catch (InterruptedException x)
{
throw new InterruptedIOException();
}
}
@Override
public void close()
{
write(Content.Chunk.EOF, Callback.NOOP);
offer(Content.Chunk.EOF, Callback.NOOP);
}
public boolean isClosed()
@ -218,7 +168,7 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
public Content.Chunk read()
{
ChunkCallback current;
try (AutoLock.WithCondition l = lock.lock())
try (AutoLock.WithCondition condition = lock.lock())
{
if (length == UNDETERMINED_LENGTH)
length = -1;
@ -233,9 +183,9 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
}
readClosed = current.chunk().isLast();
if (chunks.isEmpty())
l.signal();
condition.signal();
}
if (!current.chunk().hasRemaining())
if (!current.chunk().canRetain())
current.callback().succeeded();
return current.chunk();
}
@ -283,7 +233,7 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
public void fail(Throwable failure)
{
List<ChunkCallback> drained;
try (AutoLock ignored = lock.lock())
try (AutoLock.WithCondition condition = lock.lock())
{
if (readClosed)
return;
@ -292,6 +242,7 @@ public class AsyncContent implements Content.Sink, Content.Source, Closeable
errorChunk = Content.Chunk.from(failure);
drained = List.copyOf(chunks);
chunks.clear();
condition.signal();
}
drained.forEach(cc -> cc.callback().failed(failure));
invoker.run(this::invokeDemandCallback);

View File

@ -115,7 +115,7 @@ public class ChunksContentSource implements Content.Source
@Override
public void fail(Throwable failure)
{
List<Content.Chunk> toFail = List.of();
List<Content.Chunk> chunksToRelease;
try (AutoLock ignored = lock.lock())
{
if (terminated != null)
@ -123,10 +123,14 @@ public class ChunksContentSource implements Content.Source
terminated = Content.Chunk.from(failure);
if (iterator != null)
{
toFail = new ArrayList<>();
iterator.forEachRemaining(toFail::add);
chunksToRelease = new ArrayList<>();
iterator.forEachRemaining(chunksToRelease::add);
}
else
{
chunksToRelease = List.copyOf(chunks);
}
}
toFail.forEach(Content.Chunk::release);
chunksToRelease.forEach(Content.Chunk::release);
}
}

View File

@ -47,6 +47,9 @@ public class ContentSinkSubscriber implements Flow.Subscriber<Content.Chunk>
@Override
public void onNext(Content.Chunk chunk)
{
// Retain the chunk because the write may not complete immediately.
if (chunk.canRetain())
chunk.retain();
sink.write(chunk.isLast(), chunk.getByteBuffer(), Callback.from(() -> succeeded(chunk), x -> failed(chunk, x)));
}

View File

@ -133,6 +133,7 @@ public class ContentSourcePublisher implements Flow.Publisher<Content.Chunk>
}
subscriber.onNext(chunk);
chunk.release();
if (chunk.isLast())
{

View File

@ -82,24 +82,26 @@ public class InputStreamContentSource implements Content.Source
return Content.Chunk.EOF;
}
RetainableByteBuffer streamBuffer = bufferPool.acquire(getBufferSize(), false);
try
{
RetainableByteBuffer streamBuffer = bufferPool.acquire(getBufferSize(), false);
ByteBuffer buffer = streamBuffer.getBuffer();
int read = inputStream.read(buffer.array(), buffer.arrayOffset(), buffer.capacity());
if (read < 0)
{
streamBuffer.release();
close();
return Content.Chunk.EOF;
}
else
{
buffer.limit(read);
return Content.Chunk.from(buffer, false, streamBuffer);
return Content.Chunk.asChunk(buffer, false, streamBuffer);
}
}
catch (Throwable x)
{
streamBuffer.release();
return failure(x);
}
}

View File

@ -146,6 +146,7 @@ public class PathContentSource implements Content.Source
}
catch (Throwable x)
{
retainableByteBuffer.release();
return failure(x);
}
@ -156,7 +157,7 @@ public class PathContentSource implements Content.Source
if (last)
IO.close(channel);
return Content.Chunk.from(byteBuffer, last, retainableByteBuffer);
return Content.Chunk.asChunk(byteBuffer, last, retainableByteBuffer);
}
protected SeekableByteChannel open() throws IOException

View File

@ -62,6 +62,12 @@ public abstract class ByteBufferChunk implements Content.Chunk
return last;
}
@Override
public boolean canRetain()
{
return false;
}
@Override
public void retain()
{
@ -94,6 +100,12 @@ public abstract class ByteBufferChunk implements Content.Chunk
super(byteBuffer, last);
}
@Override
public boolean canRetain()
{
return true;
}
@Override
public void retain()
{
@ -165,6 +177,12 @@ public abstract class ByteBufferChunk implements Content.Chunk
this.retainable = retainable;
}
@Override
public boolean canRetain()
{
return true;
}
@Override
public void retain()
{

View File

@ -13,7 +13,6 @@
package org.eclipse.jetty.io;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.concurrent.CancellationException;
@ -24,13 +23,12 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.eclipse.jetty.io.content.AsyncContent;
import org.eclipse.jetty.util.BufferUtil;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.thread.Invocable;
import org.junit.jupiter.api.Test;
import static java.nio.charset.StandardCharsets.UTF_8;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.sameInstance;
@ -53,12 +51,13 @@ public class AsyncContentTest
async.demand(latch::countDown);
assertFalse(latch.await(250, TimeUnit.MILLISECONDS));
async.write(Content.Chunk.from(UTF_8.encode("one"), false), Callback.NOOP);
async.write(false, UTF_8.encode("one"), Callback.NOOP);
assertTrue(latch.await(5, TimeUnit.SECONDS));
Content.Chunk chunk = async.read();
assertNotNull(chunk);
chunk.release();
}
}
@ -77,6 +76,7 @@ public class AsyncContentTest
Content.Chunk chunk = async.read();
assertNotNull(chunk);
chunk.release();
assertTrue(chunk.isLast());
}
@ -85,10 +85,11 @@ public class AsyncContentTest
{
try (AsyncContent async = new AsyncContent())
{
async.write(Content.Chunk.from(UTF_8.encode("one"), false), Callback.NOOP);
async.write(false, UTF_8.encode("one"), Callback.NOOP);
Content.Chunk chunk = async.read();
assertNotNull(chunk);
chunk.release();
CountDownLatch latch = new CountDownLatch(1);
async.demand(latch::countDown);
@ -104,34 +105,11 @@ public class AsyncContentTest
// Offering more should fail.
CountDownLatch failLatch = new CountDownLatch(1);
async.write(Content.Chunk.EMPTY, Callback.from(Callback.NOOP::succeeded, x -> failLatch.countDown()));
async.write(false, BufferUtil.EMPTY_BUFFER, Callback.from(Callback.NOOP::succeeded, x -> failLatch.countDown()));
assertTrue(failLatch.await(5, TimeUnit.SECONDS));
}
}
@Test
public void testWriteErrorChunk() throws Exception
{
try (AsyncContent async = new AsyncContent())
{
CountDownLatch latch = new CountDownLatch(1);
async.demand(latch::countDown);
assertFalse(latch.await(250, TimeUnit.MILLISECONDS));
Throwable error = new Throwable("test");
AtomicReference<Throwable> callback = new AtomicReference<>();
async.write(Content.Chunk.from(error), Callback.from(Invocable.NOOP, callback::set));
assertThat(callback.get(), sameInstance(error));
assertTrue(latch.await(5, TimeUnit.SECONDS));
Content.Chunk chunk = async.read();
assertNotNull(chunk);
assertThat(chunk, instanceOf(Content.Chunk.Error.class));
assertThat(((Content.Chunk.Error)chunk).getCause(), sameInstance(error));
}
}
@Test
public void testChunkReleaseSucceedsWriteCallback()
{
@ -165,7 +143,8 @@ public class AsyncContentTest
Content.Chunk chunk = async.read();
assertThat(successCounter.get(), is(1));
assertThat(chunk.isTerminal(), is(false));
assertThat(chunk.isLast(), is(false));
assertThat(chunk.hasRemaining(), is(false));
assertThat(chunk.release(), is(true));
assertThat(successCounter.get(), is(1));
assertThat(failureRef.get(), is(nullValue()));
@ -184,7 +163,8 @@ public class AsyncContentTest
Content.Chunk chunk = async.read();
assertThat(successCounter.get(), is(1));
assertThat(chunk.isTerminal(), is(true));
assertThat(chunk.isLast(), is(true));
assertThat(chunk.hasRemaining(), is(false));
assertThat(chunk.release(), is(true));
assertThat(successCounter.get(), is(1));
assertThat(failureRef.get(), is(nullValue()));
@ -192,54 +172,25 @@ public class AsyncContentTest
}
@Test
public void testWriteAndReadErrors()
public void testReadAfterClose()
{
try (AsyncContent async = new AsyncContent())
{
AssertingCallback callback = new AssertingCallback();
async.close();
Exception error1 = new Exception("error1");
async.write(Content.Chunk.from(error1), callback);
callback.assertSingleFailureSameInstanceNoSuccess(error1);
Content.Chunk chunk1 = async.read();
assertThat(chunk1.isLast(), is(true));
assertThat(chunk1.hasRemaining(), is(false));
chunk1.release();
Content.Chunk chunk = async.read();
assertThat(((Content.Chunk.Error)chunk).getCause(), sameInstance(error1));
chunk = async.read();
assertThat(((Content.Chunk.Error)chunk).getCause(), sameInstance(error1));
callback.assertNoFailureNoSuccess();
Exception error2 = new Exception("error2");
async.write(Content.Chunk.from(error2), callback);
callback.assertSingleFailureSameInstanceNoSuccess(error1);
async.write(Content.Chunk.from(ByteBuffer.wrap(new byte[1]), false), callback);
callback.assertSingleFailureSameInstanceNoSuccess(error1);
Content.Chunk chunk2 = async.read();
assertThat(chunk2.isLast(), is(true));
assertThat(chunk2.hasRemaining(), is(false));
assertSame(chunk1, chunk2);
chunk2.release();
}
}
@Test
public void testCloseAfterWritingEof()
{
AssertingCallback callback = new AssertingCallback();
try (AsyncContent async = new AsyncContent())
{
async.write(Content.Chunk.EOF, callback);
callback.assertNoFailureNoSuccess();
Content.Chunk chunk = async.read();
assertThat(chunk.isTerminal(), is(true));
callback.assertNoFailureWithSuccesses(1);
chunk = async.read();
assertThat(chunk.isTerminal(), is(true));
callback.assertNoFailureWithSuccesses(0);
async.write(Content.Chunk.EOF, callback);
callback.assertSingleFailureIsInstanceNoSuccess(IOException.class);
}
callback.assertNoFailureNoSuccess();
}
@Test
public void testFailFailsCallbacks()
{
@ -318,12 +269,5 @@ public class AsyncContentTest
assertThat(throwables.size(), is(1));
assertThat(throwables.remove(0), sameInstance(x));
}
public void assertSingleFailureIsInstanceNoSuccess(Class<? extends Throwable> clazz)
{
assertThat(successCounter.get(), is(0));
assertThat(throwables.size(), is(1));
assertThat(throwables.remove(0), instanceOf(clazz));
}
}
}

View File

@ -64,8 +64,8 @@ public class ContentSourceTest
AsyncContent asyncSource = new AsyncContent();
try (asyncSource)
{
asyncSource.write(Content.Chunk.from(UTF_8.encode("one"), false), Callback.NOOP);
asyncSource.write(Content.Chunk.from(UTF_8.encode("two"), false), Callback.NOOP);
asyncSource.write(false, UTF_8.encode("one"), Callback.NOOP);
asyncSource.write(false, UTF_8.encode("two"), Callback.NOOP);
}
ByteBufferContentSource byteBufferSource = new ByteBufferContentSource(UTF_8.encode("one"), UTF_8.encode("two"));
@ -186,6 +186,7 @@ public class ContentSourceTest
Content.Chunk chunk = source.read();
assertNotNull(chunk);
chunk.release();
// Demand again, it must not recurse.
if (!chunk.isLast())
@ -209,6 +210,7 @@ public class ContentSourceTest
{
Content.Chunk chunk = source.read();
assertNotNull(chunk);
chunk.release();
if (!chunk.isLast())
{
@ -229,6 +231,7 @@ public class ContentSourceTest
{
Content.Chunk chunk = nextChunk(source);
assertNotNull(chunk);
chunk.release();
source.fail(new CancellationException());
@ -274,6 +277,7 @@ public class ContentSourceTest
Content.Chunk chunk = nextChunk(source);
assertNotNull(chunk);
chunk.release();
source.demand(() ->
{
@ -520,6 +524,9 @@ public class ContentSourceTest
private void add(Content.Chunk chunk)
{
// Retain the chunk because it is stored for later use.
if (chunk.canRetain())
chunk.retain();
_chunks.add(chunk);
}

View File

@ -54,12 +54,13 @@ public class ContentSourceTransformerTest
transformer.demand(callback::succeeded);
assertFalse(callback.isDone());
source.write(Content.Chunk.EOF, Callback.NOOP);
source.close();
assertTrue(callback.isDone());
chunk = transformer.read();
assertNotNull(chunk);
chunk.release();
assertTrue(chunk.isLast());
}
@ -68,23 +69,26 @@ public class ContentSourceTransformerTest
public void testTwoChunksAndEOF(boolean last)
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE two"), last), Callback.NOOP);
source.write(last, UTF_8.encode("ONE two"), Callback.NOOP);
if (!last)
source.write(Content.Chunk.EOF, Callback.NOOP);
source.close();
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
Content.Chunk chunk = transformer.read();
assertNotNull(chunk);
assertEquals("one", UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
chunk = transformer.read();
assertNotNull(chunk);
assertEquals("two", UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
if (last)
assertTrue(chunk.isLast());
chunk = transformer.read();
assertNotNull(chunk);
chunk.release();
assertTrue(chunk.isLast());
}
@ -92,7 +96,7 @@ public class ContentSourceTransformerTest
public void testDemandFirstWithLoop()
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE two"), true), Callback.NOOP);
source.write(true, UTF_8.encode("ONE two"), Callback.NOOP);
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
AtomicBoolean processed = new AtomicBoolean();
@ -103,6 +107,7 @@ public class ContentSourceTransformerTest
{
Content.Chunk chunk = transformer.read();
assertNotNull(chunk);
chunk.release();
if (chunk.isLast())
break;
}
@ -115,7 +120,7 @@ public class ContentSourceTransformerTest
public void testDemandFirstWithoutLoop()
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE NOOP two"), true), Callback.NOOP);
source.write(true, UTF_8.encode("ONE NOOP two"), Callback.NOOP);
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
AtomicBoolean reEnter = new AtomicBoolean();
@ -131,6 +136,7 @@ public class ContentSourceTransformerTest
Content.Chunk chunk = transformer.read();
assertNotNull(chunk);
assertEquals(expected.poll(), UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
if (!chunk.isLast())
transformer.demand(this);
@ -147,7 +153,7 @@ public class ContentSourceTransformerTest
public void testDemandFirstWithoutLoopStallAfterTwoExpectedChunks()
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE NOOP two"), false), Callback.NOOP);
source.write(false, UTF_8.encode("ONE NOOP two"), Callback.NOOP);
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
AtomicBoolean reEnter = new AtomicBoolean();
@ -162,7 +168,10 @@ public class ContentSourceTransformerTest
Content.Chunk chunk = transformer.read();
if (chunk != null)
{
assertEquals(expected.poll(), UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
}
if (chunk == null || !chunk.isLast())
transformer.demand(this);
@ -175,7 +184,7 @@ public class ContentSourceTransformerTest
assertThat(expected, empty());
expected.offer("three");
source.write(Content.Chunk.from(UTF_8.encode("three"), true), Callback.NOOP);
source.write(true, UTF_8.encode("three"), Callback.NOOP);
assertThat(expected, empty());
expected.offer("EOF");
@ -184,6 +193,7 @@ public class ContentSourceTransformerTest
Content.Chunk chunk = transformer.read();
assertTrue(chunk.isLast());
assertFalse(chunk.hasRemaining());
chunk.release();
expected.poll();
});
@ -194,7 +204,7 @@ public class ContentSourceTransformerTest
public void testDemandFirstThenConsumeAllChunks()
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE NOOP two"), true), Callback.NOOP);
source.write(true, UTF_8.encode("ONE NOOP two"), Callback.NOOP);
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
AtomicInteger count = new AtomicInteger();
@ -213,6 +223,7 @@ public class ContentSourceTransformerTest
Content.Chunk chunk = transformer.read();
assertNotNull(chunk);
assertEquals("one", UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
// This demand will be fulfilled later after the last chunk has been read.
transformer.demand(this);
@ -221,6 +232,7 @@ public class ContentSourceTransformerTest
assertNotNull(chunk);
assertEquals("two", UTF_8.decode(chunk.getByteBuffer()).toString());
assertTrue(chunk.isLast());
chunk.release();
if (!reEnter.compareAndSet(true, false))
throw new IllegalStateException();
@ -236,14 +248,15 @@ public class ContentSourceTransformerTest
public void testTransformThrows()
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE"), false), Callback.NOOP);
source.write(Content.Chunk.from(UTF_8.encode("THROW"), false), Callback.NOOP);
source.write(Content.Chunk.from(UTF_8.encode("two"), true), Callback.NOOP);
source.write(false, UTF_8.encode("ONE"), Callback.NOOP);
source.write(false, UTF_8.encode("THROW"), Callback.NOOP);
source.write(true, UTF_8.encode("two"), Callback.NOOP);
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
Content.Chunk chunk = transformer.read();
assertNotNull(chunk);
assertEquals("one", UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
chunk = transformer.read();
assertInstanceOf(Content.Chunk.Error.class, chunk);
@ -260,14 +273,15 @@ public class ContentSourceTransformerTest
public void testTransformReturnsError()
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE"), false), Callback.NOOP);
source.write(Content.Chunk.from(UTF_8.encode("ERROR"), false), Callback.NOOP);
source.write(Content.Chunk.from(UTF_8.encode("two"), true), Callback.NOOP);
source.write(false, UTF_8.encode("ONE"), Callback.NOOP);
source.write(false, UTF_8.encode("ERROR"), Callback.NOOP);
source.write(true, UTF_8.encode("two"), Callback.NOOP);
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
Content.Chunk chunk = transformer.read();
assertNotNull(chunk);
assertEquals("one", UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
chunk = transformer.read();
assertInstanceOf(Content.Chunk.Error.class, chunk);
@ -281,13 +295,15 @@ public class ContentSourceTransformerTest
public void testSourceReturnsError()
{
AsyncContent source = new AsyncContent();
source.write(Content.Chunk.from(UTF_8.encode("ONE"), false), Callback.NOOP);
source.write(Content.Chunk.from(new IOException()), Callback.NOOP);
source.write(false, UTF_8.encode("ONE"), Callback.NOOP);
WordSplitLowCaseTransformer transformer = new WordSplitLowCaseTransformer(source);
Content.Chunk chunk = transformer.read();
assertNotNull(chunk);
assertEquals("one", UTF_8.decode(chunk.getByteBuffer()).toString());
chunk.release();
source.fail(new IOException());
chunk = transformer.read();
assertInstanceOf(Content.Chunk.Error.class, chunk);
@ -333,7 +349,6 @@ public class ContentSourceTransformerTest
if (rawChunk.isLast() && !last)
chunks.offer(Content.Chunk.EOF);
}
return chunks.poll();
}
}

View File

@ -66,13 +66,13 @@ public class ContentTest
{
Retainable.ReferenceCounter referenceCounter1 = new Retainable.ReferenceCounter(2);
assertThat(referenceCounter1.isRetained(), is(true));
assertThat(Content.Chunk.from(ByteBuffer.wrap(new byte[0]), true, referenceCounter1), sameInstance(Content.Chunk.EOF));
assertThat(Content.Chunk.asChunk(ByteBuffer.wrap(new byte[0]), true, referenceCounter1), sameInstance(Content.Chunk.EOF));
assertThat(referenceCounter1.isRetained(), is(false));
assertThat(referenceCounter1.release(), is(true));
Retainable.ReferenceCounter referenceCounter2 = new Retainable.ReferenceCounter(2);
assertThat(referenceCounter2.isRetained(), is(true));
assertThat(Content.Chunk.from(ByteBuffer.wrap(new byte[0]), false, referenceCounter2), sameInstance(Content.Chunk.EMPTY));
assertThat(Content.Chunk.asChunk(ByteBuffer.wrap(new byte[0]), false, referenceCounter2), sameInstance(Content.Chunk.EMPTY));
assertThat(referenceCounter2.isRetained(), is(false));
assertThat(referenceCounter2.release(), is(true));
}

View File

@ -248,7 +248,7 @@ public abstract class ProxyHandler extends Handler.Abstract
LOG.debug("{} P2S continuing request", requestId(clientToProxyRequest));
var proxyToServerRequestContent = newProxyToServerRequestContent(clientToProxyRequest, proxyToClientResponse, proxyToServerRequest);
Content.copy(proxyToServerRequestContent, delayedProxyToServerRequestContent,
Callback.from(delayedProxyToServerRequestContent::close, x -> delayedProxyToServerRequestContent.write(Content.Chunk.from(x), Callback.NOOP)));
Callback.from(delayedProxyToServerRequestContent::close, delayedProxyToServerRequestContent::fail));
};
proxyToServerRequest.attribute(PROXY_TO_SERVER_CONTINUE_ATTRIBUTE, action);
}
@ -687,6 +687,9 @@ public abstract class ProxyHandler extends Handler.Abstract
ByteBuffer serverToProxyContent = serverToProxyChunk.getByteBuffer();
if (LOG.isDebugEnabled())
LOG.debug("{} S2P received content {}", requestId(clientToProxyRequest), BufferUtil.toDetailString(serverToProxyContent));
if (serverToProxyChunk.canRetain())
serverToProxyChunk.retain();
Callback callback = new Callback()
{
@Override
@ -714,6 +717,7 @@ public abstract class ProxyHandler extends Handler.Abstract
return InvocationType.NON_BLOCKING;
}
};
proxyToClientResponse.write(false, serverToProxyContent, callback);
}

View File

@ -138,11 +138,12 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
@Override
public void run()
{
Content.Chunk chunk = null;
try
{
while (true)
{
Content.Chunk chunk = _source.read();
chunk = _source.read();
if (chunk == null)
{
_source.demand(this);
@ -163,6 +164,8 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
if (_maxFields >= 0 && _fields.getSize() >= _maxFields)
{
chunk.release();
// Do not double release if completeExceptionally() throws.
chunk = null;
completeExceptionally(new IllegalStateException("form with too many fields"));
return;
}
@ -172,6 +175,8 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
chunk.release();
if (chunk.isLast())
{
// Do not double release if complete() throws.
chunk = null;
complete(_fields);
return;
}
@ -179,6 +184,8 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
}
catch (Throwable x)
{
if (chunk != null)
chunk.release();
completeExceptionally(x);
}
}
@ -214,7 +221,7 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
case '=' ->
{
_name = _builder.takeString();
checkLength(chunk, _name);
checkLength(_name);
}
case '+' -> _builder.append((byte)' ');
case '%' -> _percent++;
@ -228,7 +235,7 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
case '&' ->
{
value = _builder.takeString();
checkLength(chunk, value);
checkLength(value);
break loop;
}
case '+' -> _builder.append((byte)' ');
@ -248,7 +255,7 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
_builder.append(_percentCode);
}
value = _builder.takeString();
checkLength(chunk, value);
checkLength(value);
}
if (value != null)
@ -262,16 +269,13 @@ public class FormFields extends CompletableFuture<Fields> implements Runnable
return null;
}
private void checkLength(Content.Chunk chunk, String nameOrValue)
private void checkLength(String nameOrValue)
{
if (_maxLength >= 0)
{
_length += nameOrValue.length();
if (_length > _maxLength)
{
chunk.release();
throw new IllegalStateException("form too large");
}
}
}
}

View File

@ -180,13 +180,16 @@ public class DelayedHandler extends Handler.Wrapper
}
else
{
RewindChunkRequest request = new RewindChunkRequest(getRequest(), chunk);
try
{
getHandler().process(new RewindChunkRequest(getRequest(), chunk), getResponse(), getCallback());
getHandler().process(request, getResponse(), getCallback());
}
catch (Exception e)
catch (Throwable x)
{
Response.writeError(getRequest(), getResponse(), getCallback(), e);
// Use the wrapped request so that the error handling can
// consume the request content and release the already read chunk.
Response.writeError(request, getResponse(), getCallback(), x);
}
}
}

View File

@ -165,7 +165,7 @@ public class GzipRequest extends Request.Wrapper
return Content.Chunk.EOF;
// Retain the input chunk because its ByteBuffer will be referenced by the Inflater.
if (retain)
if (retain && _chunk.canRetain())
_chunk.retain();
ByteBuffer decodedBuffer = _decoder.decode(_chunk);
@ -177,6 +177,7 @@ public class GzipRequest extends Request.Wrapper
}
else
{
_decoder.release(decodedBuffer);
// Could not decode more from this chunk, release it.
Content.Chunk result = _chunk.isLast() ? Content.Chunk.EOF : null;
_chunk.release();
@ -195,12 +196,12 @@ public class GzipRequest extends Request.Wrapper
super(inflaterPool, bufferPool, bufferSize);
}
public ByteBuffer decode(Content.Chunk content)
public ByteBuffer decode(Content.Chunk chunk)
{
decodeChunks(content.getByteBuffer());
ByteBuffer chunk = _decoded;
decodeChunks(chunk.getByteBuffer());
ByteBuffer decoded = _decoded;
_decoded = null;
return chunk;
return decoded;
}
@Override

View File

@ -51,7 +51,6 @@ import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.EofException;
import org.eclipse.jetty.io.Retainable;
import org.eclipse.jetty.io.RetainableByteBuffer;
import org.eclipse.jetty.io.RetainableByteBufferPool;
import org.eclipse.jetty.io.WriteFlusher;
@ -988,13 +987,11 @@ public class HttpConnection extends AbstractConnection implements Runnable, Writ
if (stream == null || stream._chunk != null || _retainableByteBuffer == null)
throw new IllegalStateException();
_retainableByteBuffer.retain();
if (LOG.isDebugEnabled())
LOG.debug("content {}/{} for {}", BufferUtil.toDetailString(buffer), _retainableByteBuffer, HttpConnection.this);
RetainableByteBuffer retainable = _retainableByteBuffer;
stream._chunk = Content.Chunk.from(buffer, false, new ChunkRetainable(retainable, buffer));
_retainableByteBuffer.retain();
stream._chunk = Content.Chunk.asChunk(buffer, false, _retainableByteBuffer);
return true;
}
@ -1109,26 +1106,6 @@ public class HttpConnection extends AbstractConnection implements Runnable, Writ
}
}
private class ChunkRetainable extends Retainable.Wrapper
{
private final ByteBuffer buffer;
private ChunkRetainable(Retainable retainable, ByteBuffer buffer)
{
super(retainable);
this.buffer = buffer;
}
@Override
public boolean release()
{
boolean released = super.release();
if (LOG.isDebugEnabled())
LOG.debug("content released {} {}/{} for {}", released, BufferUtil.toDetailString(buffer), getWrapped(), HttpConnection.this);
return released;
}
}
protected class HttpStreamOverHTTP1 implements HttpStream
{
private final long _nanoTime = NanoTime.now();

View File

@ -481,11 +481,12 @@ public abstract class HttpServerTestBase extends HttpServerTestFixture
{
read += chunk.remaining();
chunk.getByteBuffer().clear();
chunk.release();
if (!fourBytesRead.get() && read >= 4)
fourBytesRead.set(true);
}
chunk.release();
if (chunk.isLast())
{
callback.succeeded();
@ -1708,6 +1709,8 @@ public abstract class HttpServerTestBase extends HttpServerTestFixture
if (chunk.hasRemaining())
contents.add(chunk);
else
chunk.release();
if (chunk.isLast())
break;

View File

@ -166,9 +166,10 @@ public class HttpServerTestFixture
int r = c.remaining();
c.get(content, offset, r);
offset += r;
c.release();
}
c.release();
if (c.isLast())
break;
}

View File

@ -47,6 +47,12 @@ public class MockHttpStream implements HttpStream
return false;
}
@Override
public boolean canRetain()
{
return false;
}
@Override
public void retain()
{
@ -98,8 +104,10 @@ public class MockHttpStream implements HttpStream
return addContent(BufferUtil.toBuffer(content), last);
}
public Runnable addContent(Content.Chunk chunk)
private Runnable addContent(Content.Chunk chunk)
{
if (chunk.canRetain())
chunk.retain();
chunk = _content.getAndSet(chunk);
if (chunk == DEMAND)
return _channel.onContentAvailable();

View File

@ -316,7 +316,8 @@ public class ContextHandlerTest
},
t ->
{
throw new IllegalStateException();
chunk.release();
throw new IllegalStateException(t);
}));
});
return true;

View File

@ -118,10 +118,11 @@ public class DumpHandler extends Handler.Abstract
if (!chunk.hasRemaining())
{
boolean last = chunk.isLast();
chunk.release();
if (chunk.isLast())
break;
chunk = null;
if (last)
break;
}
}
if (chunk != null)

View File

@ -95,10 +95,9 @@ public class SSLReadEOFAfterResponseTest
continue;
}
if (c.hasRemaining())
{
length -= c.remaining();
c.release();
}
c.release();
// TODO: should not compare to EOF.
if (c == Content.Chunk.EOF)
callback.failed(new IllegalStateException());
}
@ -115,6 +114,7 @@ public class SSLReadEOFAfterResponseTest
// Third, read the EOF.
Content.Chunk chunk = request.read();
chunk.release();
if (!chunk.isLast())
throw new IllegalStateException();
callback.succeeded();

View File

@ -98,7 +98,6 @@ public class HttpClientDemandTest extends AbstractTest
@Override
public void onContent(Response response, Content.Chunk chunk, Runnable demander)
{
chunk.release();
if (chunks.incrementAndGet() == 1)
contentLatch.countDown();
// Need to demand also after the second
@ -154,7 +153,9 @@ public class HttpClientDemandTest extends AbstractTest
@Override
public void onContent(Response response, Content.Chunk chunk, Runnable demander)
{
// Don't demand and don't release chunks.
// Store the chunk and don't demand.
if (chunk.canRetain())
chunk.retain();
contentQueue.offer(chunk);
demanderQueue.offer(demander);
}
@ -243,6 +244,8 @@ public class HttpClientDemandTest extends AbstractTest
client.newRequest(newURI(transport))
.onResponseContentAsync((response, chunk, demander) ->
{
if (chunk.canRetain())
chunk.retain();
chunkRef.set(chunk);
try
{
@ -332,7 +335,6 @@ public class HttpClientDemandTest extends AbstractTest
{
listener1Chunks.incrementAndGet();
listener1ContentSize.addAndGet(chunk.remaining());
chunk.release();
listener1DemanderRef.set(demander);
};
AtomicInteger listener2Chunks = new AtomicInteger();
@ -342,7 +344,6 @@ public class HttpClientDemandTest extends AbstractTest
{
listener2Chunks.incrementAndGet();
listener2ContentSize.addAndGet(chunk.remaining());
chunk.release();
listener2DemanderRef.set(demander);
};
@ -414,9 +415,9 @@ public class HttpClientDemandTest extends AbstractTest
client.newRequest(newURI(transport))
.onResponseContentAsync((response, chunk, demander) ->
{
boolean demand = chunk.hasRemaining();
received.put(chunk.getByteBuffer());
chunk.release();
if (!chunk.isTerminal())
if (demand)
new Thread(demander).start();
})
.send(result ->
@ -472,7 +473,7 @@ public class HttpClientDemandTest extends AbstractTest
demander.run();
return;
}
if (chunk.isTerminal())
if (chunk.isLast() && !chunk.hasRemaining())
{
chunk.release();
return;
@ -533,7 +534,7 @@ public class HttpClientDemandTest extends AbstractTest
demander.run();
return;
}
if (chunk.isTerminal())
if (chunk.isLast() && !chunk.hasRemaining())
{
chunk.release();
return;

View File

@ -516,7 +516,6 @@ public class HttpClientTest extends AbstractTest
client.newRequest(newURI(transport))
.onResponseContentAsync((response, chunk, demander) ->
{
chunk.release();
if (counter.incrementAndGet() == 1)
{
demanderRef.set(demander);
@ -827,7 +826,7 @@ public class HttpClientTest extends AbstractTest
@Override
public void onContentSource(Response response, Content.Source contentSource)
{
accumulateChunks(response, contentSource, chunks);
accumulateChunks(contentSource, chunks);
}
};
client.newRequest(newURI(transport))
@ -877,7 +876,7 @@ public class HttpClientTest extends AbstractTest
@Override
public void onContentSource(Response response, Content.Source contentSource)
{
new Thread(() -> accumulateChunksInSpawnedThread(response, contentSource, chunks))
new Thread(() -> accumulateChunksInSpawnedThread(contentSource, chunks))
.start();
}
};
@ -904,9 +903,9 @@ public class HttpClientTest extends AbstractTest
ContentResponse resp = client.newRequest(newURI(transport))
.path("/")
.onResponseContentSource((response, contentSource) -> accumulateChunks(response, contentSource, chunks1))
.onResponseContentSource((response, contentSource) -> accumulateChunks(response, contentSource, chunks2))
.onResponseContentSource((response, contentSource) -> accumulateChunks(response, contentSource, chunks3))
.onResponseContentSource((response, contentSource) -> accumulateChunks(contentSource, chunks1))
.onResponseContentSource((response, contentSource) -> accumulateChunks(contentSource, chunks2))
.onResponseContentSource((response, contentSource) -> accumulateChunks(contentSource, chunks3))
.send();
assertThat(resp.getStatus(), is(200));
@ -932,8 +931,8 @@ public class HttpClientTest extends AbstractTest
List<Content.Chunk> chunks3 = new CopyOnWriteArrayList<>();
ContentResponse contentResponse = client.newRequest(newURI(transport))
.path("/")
.onResponseContentSource((response, contentSource) -> accumulateChunks(response, contentSource, chunks1))
.onResponseContentSource((response, contentSource) -> accumulateChunks(response, contentSource, chunks2))
.onResponseContentSource((response, contentSource) -> accumulateChunks(contentSource, chunks1))
.onResponseContentSource((response, contentSource) -> accumulateChunks(contentSource, chunks2))
.onResponseContentSource((response, contentSource) ->
{
contentSource.fail(new Exception("Synthetic Failure"));
@ -948,6 +947,10 @@ public class HttpClientTest extends AbstractTest
assertThat(chunks3.stream().mapToInt(c -> c.getByteBuffer().remaining()).sum(), is(0));
assertThat(chunks3.size(), is(1));
assertThat(chunks3.get(0), instanceOf(Content.Chunk.Error.class));
chunks1.forEach(Content.Chunk::release);
chunks2.forEach(Content.Chunk::release);
chunks3.forEach(Content.Chunk::release);
}
@ParameterizedTest
@ -963,8 +966,8 @@ public class HttpClientTest extends AbstractTest
CountDownLatch chunks3Latch = new CountDownLatch(1);
ContentResponse contentResponse = client.newRequest(newURI(transport))
.path("/")
.onResponseContentSource((response, contentSource) -> accumulateChunks(response, contentSource, chunks1))
.onResponseContentSource((response, contentSource) -> accumulateChunks(response, contentSource, chunks2))
.onResponseContentSource((response, contentSource) -> accumulateChunks(contentSource, chunks1))
.onResponseContentSource((response, contentSource) -> accumulateChunks(contentSource, chunks2))
.onResponseContentSource((response, contentSource) ->
new Thread(() ->
{
@ -986,6 +989,10 @@ public class HttpClientTest extends AbstractTest
assertThat(chunks3.stream().mapToInt(c -> c.getByteBuffer().remaining()).sum(), is(0));
assertThat(chunks3.size(), is(1));
assertThat(chunks3.get(0), instanceOf(Content.Chunk.Error.class));
chunks1.forEach(Content.Chunk::release);
chunks2.forEach(Content.Chunk::release);
chunks3.forEach(Content.Chunk::release);
}
@ParameterizedTest
@ -1025,47 +1032,52 @@ public class HttpClientTest extends AbstractTest
}
}
private static void accumulateChunks(Response response, Content.Source contentSource, List<Content.Chunk> chunks)
private static void accumulateChunks(Content.Source contentSource, List<Content.Chunk> chunks)
{
Content.Chunk chunk = contentSource.read();
if (chunk == null)
{
contentSource.demand(() -> accumulateChunks(response, contentSource, chunks));
contentSource.demand(() -> accumulateChunks(contentSource, chunks));
return;
}
chunks.add(duplicateAndRelease(chunk));
if (!chunk.isLast())
contentSource.demand(() -> accumulateChunks(response, contentSource, chunks));
}
private static void accumulateChunksInSpawnedThread(Response response, Content.Source contentSource, List<Content.Chunk> chunks)
{
Content.Chunk chunk = contentSource.read();
if (chunk == null)
{
contentSource.demand(() -> new Thread(() -> accumulateChunks(response, contentSource, chunks)).start());
return;
}
chunks.add(duplicateAndRelease(chunk));
if (!chunk.isLast())
contentSource.demand(() -> new Thread(() -> accumulateChunks(response, contentSource, chunks)).start());
}
private static Content.Chunk duplicateAndRelease(Content.Chunk chunk)
{
if (chunk == null || chunk.isTerminal())
return chunk;
ByteBuffer buffer = BufferUtil.allocate(chunk.remaining());
int pos = BufferUtil.flipToFill(buffer);
buffer.put(chunk.getByteBuffer());
BufferUtil.flipToFlush(buffer, pos);
chunks.add(duplicate(chunk));
chunk.release();
return Content.Chunk.from(buffer, chunk.isLast());
if (!chunk.isLast())
contentSource.demand(() -> accumulateChunks(contentSource, chunks));
}
private static void accumulateChunksInSpawnedThread(Content.Source contentSource, List<Content.Chunk> chunks)
{
Content.Chunk chunk = contentSource.read();
if (chunk == null)
{
contentSource.demand(() -> new Thread(() -> accumulateChunks(contentSource, chunks)).start());
return;
}
chunks.add(duplicate(chunk));
chunk.release();
if (!chunk.isLast())
contentSource.demand(() -> new Thread(() -> accumulateChunks(contentSource, chunks)).start());
}
private static Content.Chunk duplicate(Content.Chunk chunk)
{
if (chunk.hasRemaining())
{
ByteBuffer byteBuffer = BufferUtil.allocate(chunk.remaining());
int pos = BufferUtil.flipToFill(byteBuffer);
byteBuffer.put(chunk.getByteBuffer());
BufferUtil.flipToFlush(byteBuffer, pos);
return Content.Chunk.from(byteBuffer, chunk.isLast());
}
else
{
return chunk.isLast() ? Content.Chunk.EOF : Content.Chunk.EMPTY;
}
}
private static class TestProcessor extends Handler.Abstract

View File

@ -439,6 +439,8 @@ public class AsyncMiddleManServlet extends AbstractProxyServlet
@Override
public void onContent(Response serverResponse, Content.Chunk chunk, Runnable demander)
{
if (chunk.canRetain())
chunk.retain();
Callback callback = Callback.from(chunk::release, Callback.from(demander, serverResponse::abort));
try
{

View File

@ -216,6 +216,8 @@ public class ProxyServlet extends AbstractProxyServlet
content.get(buffer);
offset = 0;
}
if (chunk.canRetain())
chunk.retain();
Callback callback = Callback.from(chunk::release, Callback.from(demander, proxyResponse::abort));
onResponseContent(request, response, proxyResponse, buffer, offset, length, callback);
}

View File

@ -58,11 +58,12 @@ class AsyncContentProducer implements ContentProducer
if (LOG.isDebugEnabled())
LOG.debug("recycling {}", this);
// Make sure that the chunk has been fully consumed before destroying the interceptor and also make sure
// that asking this instance for chunks between recycle and reopen will only produce error'ed chunks.
// Make sure that asking this instance for chunks between
// recycle() and reopen() will only produce error chunks.
if (_chunk == null)
_chunk = RECYCLED_ERROR_CHUNK;
else if (!_chunk.isTerminal())
// The chunk must be fully consumed.
else if (!_chunk.isLast() || _chunk.hasRemaining())
throw new IllegalStateException("ContentProducer with unconsumed chunk cannot be recycled");
}
@ -168,15 +169,14 @@ class AsyncContentProducer implements ContentProducer
private boolean consumeCurrentChunk()
{
if (_chunk != null && !_chunk.isTerminal())
if (_chunk != null)
{
if (LOG.isDebugEnabled())
LOG.debug("releasing current chunk {}", this);
LOG.debug("consuming and releasing current chunk {}", this);
_chunk.skip(_chunk.remaining());
_chunk.release();
_chunk = _chunk.isLast() ? Content.Chunk.EOF : null;
}
return _chunk != null && _chunk.isLast();
}
@ -191,8 +191,8 @@ class AsyncContentProducer implements ContentProducer
chunk.release();
if (chunk.isTerminal())
return chunk.isLast();
if (chunk.isLast())
return true;
}
}
@ -266,7 +266,7 @@ class AsyncContentProducer implements ContentProducer
{
if (_chunk != null)
{
if (_chunk.isTerminal() || _chunk.hasRemaining())
if (_chunk.isLast() || _chunk.hasRemaining())
{
if (LOG.isDebugEnabled())
LOG.debug("chunk not yet depleted, returning it {}", this);
@ -276,7 +276,6 @@ class AsyncContentProducer implements ContentProducer
{
if (LOG.isDebugEnabled())
LOG.debug("current chunk depleted {}", this);
_chunk.release();
_chunk = null;
}
@ -299,7 +298,7 @@ class AsyncContentProducer implements ContentProducer
}
// Release the chunk immediately, if it is empty.
if (_chunk != null && !_chunk.hasRemaining() && !_chunk.isTerminal())
if (_chunk != null && !_chunk.hasRemaining() && !_chunk.isLast())
{
if (LOG.isDebugEnabled())
LOG.debug("releasing empty chunk {}", this);

View File

@ -245,7 +245,7 @@ public class HttpInput extends ServletInputStream implements Runnable
Content.Chunk chunk = _contentProducer.nextChunk();
if (chunk == null)
throw new IllegalStateException("read on unready input");
if (!chunk.isTerminal())
if (chunk.hasRemaining())
{
int read = buffer == null ? get(chunk, b, off, len) : get(chunk, buffer);
if (LOG.isDebugEnabled())
@ -343,31 +343,28 @@ public class HttpInput extends ServletInputStream implements Runnable
return;
}
if (chunk.isTerminal())
if (chunk instanceof Content.Chunk.Error errorChunk)
{
if (chunk instanceof Content.Chunk.Error errorChunk)
Throwable error = errorChunk.getCause();
if (LOG.isDebugEnabled())
LOG.debug("running error={} {}", error, this);
// TODO is this necessary to add here?
_servletChannel.getResponse().getHeaders().add(HttpFields.CONNECTION_CLOSE);
_readListener.onError(error);
}
else if (chunk.isLast() && !chunk.hasRemaining())
{
try
{
Throwable error = errorChunk.getCause();
if (LOG.isDebugEnabled())
LOG.debug("running error={} {}", error, this);
// TODO is this necessary to add here?
_servletChannel.getResponse().getHeaders().add(HttpFields.CONNECTION_CLOSE);
_readListener.onError(error);
LOG.debug("running at EOF {}", this);
_readListener.onAllDataRead();
}
else if (chunk == Content.Chunk.EOF)
catch (Throwable x)
{
try
{
if (LOG.isDebugEnabled())
LOG.debug("running at EOF {}", this);
_readListener.onAllDataRead();
}
catch (Throwable x)
{
if (LOG.isDebugEnabled())
LOG.debug("running failed onAllDataRead {}", this, x);
_readListener.onError(x);
}
if (LOG.isDebugEnabled())
LOG.debug("running failed onAllDataRead {}", this, x);
_readListener.onError(x);
}
}
else

View File

@ -111,7 +111,9 @@ public class ServletMultiPartFormData
formData.parse(Content.Chunk.EOF);
break;
}
formData.parse(Content.Chunk.from(ByteBuffer.wrap(buffer, 0, read), false));
Content.Chunk chunk = Content.Chunk.from(ByteBuffer.wrap(buffer, 0, read), false);
formData.parse(chunk);
chunk.release();
}
return new Parts(formData);

View File

@ -600,7 +600,6 @@ public class ServerTimeoutsTest extends AbstractTest
.onResponseContentAsync((response, chunk, demander) ->
{
objects.offer(chunk.remaining());
chunk.release();
objects.offer(demander);
})
.send(result ->

View File

@ -19,9 +19,6 @@ import java.util.concurrent.locks.Condition;
import org.eclipse.jetty.http.BadMessageException;
import org.eclipse.jetty.http.HttpStatus;
import org.eclipse.jetty.http.Trailers;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.server.Response;
import org.eclipse.jetty.util.NanoTime;
import org.eclipse.jetty.util.StaticException;
import org.eclipse.jetty.util.component.Destroyable;
@ -36,13 +33,14 @@ import org.slf4j.LoggerFactory;
class AsyncContentProducer implements ContentProducer
{
private static final Logger LOG = LoggerFactory.getLogger(AsyncContentProducer.class);
private static final Content.Chunk.Error RECYCLED_ERROR_CONTENT = Content.Chunk.from(new StaticException("ContentProducer has been recycled"));
private static final HttpInput.ErrorContent RECYCLED_ERROR_CONTENT = new HttpInput.ErrorContent(new StaticException("ContentProducer has been recycled"));
private static final Throwable UNCONSUMED_CONTENT_EXCEPTION = new StaticException("Unconsumed content");
private final AutoLock _lock = new AutoLock();
private final HttpChannel _httpChannel;
private HttpInput.Interceptor _interceptor;
private Content.Chunk _rawContent;
private Content.Chunk _transformedContent;
private HttpInput.Content _rawContent;
private HttpInput.Content _transformedContent;
private boolean _error;
private long _firstByteNanoTime = Long.MIN_VALUE;
private long _rawContentArrived;
@ -69,12 +67,12 @@ class AsyncContentProducer implements ContentProducer
// that asking this instance for content between recycle and reopen will only produce error'ed content.
if (_rawContent == null)
_rawContent = RECYCLED_ERROR_CONTENT;
else if (!_rawContent.isTerminal())
else if (!_rawContent.isSpecial())
throw new IllegalStateException("ContentProducer with unconsumed content cannot be recycled");
if (_transformedContent == null)
_transformedContent = RECYCLED_ERROR_CONTENT;
else if (!_transformedContent.isTerminal())
else if (!_transformedContent.isSpecial())
throw new IllegalStateException("ContentProducer with unconsumed content cannot be recycled");
if (_interceptor instanceof Destroyable)
@ -113,7 +111,7 @@ class AsyncContentProducer implements ContentProducer
public int available()
{
assertLocked();
Content.Chunk content = nextTransformedContent();
HttpInput.Content content = nextTransformedContent();
int available = content == null ? 0 : content.remaining();
if (LOG.isDebugEnabled())
LOG.debug("available = {} {}", available, this);
@ -164,7 +162,7 @@ class AsyncContentProducer implements ContentProducer
LOG.debug("checkMinDataRate aborting channel {}", this);
_httpChannel.abort(bad);
}
releaseContent();
failCurrentContent(bad);
throw bad;
}
}
@ -184,50 +182,56 @@ class AsyncContentProducer implements ContentProducer
public boolean consumeAll()
{
assertLocked();
// release any raw content we have already and check for EOF
boolean eof = releaseContent();
if (eof)
return true;
while (true)
Throwable x = UNCONSUMED_CONTENT_EXCEPTION;
if (LOG.isTraceEnabled())
{
_rawContent = _httpChannel.getCoreRequest().read();
if (_rawContent == null)
return false;
_rawContent.release();
if (_rawContent.isLast())
return true;
x = new StaticException("Unconsumed content", true);
LOG.trace("consumeAll {}", this, x);
}
failCurrentContent(x);
// A specific HttpChannel mechanism must be used as the following code
// does not guarantee that the channel will synchronously deliver all
// content it already contains:
// while (true)
// {
// HttpInput.Content content = _httpChannel.produceContent();
// ...
// }
// as the HttpChannel's produceContent() contract makes no such promise;
// for instance the H2 implementation calls Stream.demand() that may
// deliver the content asynchronously. Tests in StreamResetTest cover this.
boolean atEof = _httpChannel.failAllContent(x);
if (LOG.isDebugEnabled())
LOG.debug("failed all content of http channel EOF={} {}", atEof, this);
return atEof;
}
@Override
public boolean releaseContent()
private void failCurrentContent(Throwable x)
{
assertLocked();
// release transformed content, ignoring any unconsumed.
if (_transformedContent != null && !_transformedContent.isTerminal())
if (_transformedContent != null && !_transformedContent.isSpecial())
{
if (_transformedContent != _rawContent)
{
if (LOG.isDebugEnabled())
LOG.debug("failing currently held transformed content {} {}", null, this);
LOG.debug("failing currently held transformed content {} {}", x, this);
_transformedContent.skip(_transformedContent.remaining());
_transformedContent.release();
_transformedContent.failed(x);
}
_transformedContent = null;
}
if (_rawContent != null && !_rawContent.isTerminal())
if (_rawContent != null && !_rawContent.isSpecial())
{
if (LOG.isDebugEnabled())
LOG.debug("failing currently held raw content {} {}", null, this);
_rawContent.release();
_rawContent = _rawContent.isLast() ? Content.Chunk.EOF : null;
LOG.debug("failing currently held raw content {} {}", x, this);
_rawContent.skip(_rawContent.remaining());
_rawContent.failed(x);
_rawContent = null;
}
return _rawContent != null && _rawContent.isLast();
HttpInput.ErrorContent errorContent = new HttpInput.ErrorContent(x);
_transformedContent = errorContent;
_rawContent = errorContent;
}
@Override
@ -240,10 +244,10 @@ class AsyncContentProducer implements ContentProducer
}
@Override
public Content.Chunk nextContent()
public HttpInput.Content nextContent()
{
assertLocked();
Content.Chunk content = nextTransformedContent();
HttpInput.Content content = nextTransformedContent();
if (LOG.isDebugEnabled())
LOG.debug("nextContent = {} {}", content, this);
if (content != null)
@ -252,14 +256,14 @@ class AsyncContentProducer implements ContentProducer
}
@Override
public void reclaim(Content.Chunk content)
public void reclaim(HttpInput.Content content)
{
assertLocked();
if (LOG.isDebugEnabled())
LOG.debug("reclaim {} {}", content, this);
if (_transformedContent == content)
{
content.release();
content.succeeded();
if (_transformedContent == _rawContent)
_rawContent = null;
_transformedContent = null;
@ -270,7 +274,7 @@ class AsyncContentProducer implements ContentProducer
public boolean isReady()
{
assertLocked();
Content.Chunk content = nextTransformedContent();
HttpInput.Content content = nextTransformedContent();
if (content != null)
{
if (LOG.isDebugEnabled())
@ -279,12 +283,25 @@ class AsyncContentProducer implements ContentProducer
}
_httpChannel.getState().onReadUnready();
_httpChannel.getCoreRequest().demand(() ->
while (_httpChannel.needContent())
{
if (_httpChannel.getRequest().getHttpInput().onContentProducible())
_httpChannel.handle();
});
content = nextTransformedContent();
if (LOG.isDebugEnabled())
LOG.debug("isReady(), got transformed content after needContent retry {} {}", content, this);
if (content != null)
{
_httpChannel.getState().onContentAdded();
return true;
}
else
{
// We could have read some rawContent but not enough to generate
// transformed content, so we need to call needContent() again
// to tell the channel that more content is needed.
if (LOG.isDebugEnabled())
LOG.debug("isReady(), could not transform content after needContent retry {}", this);
}
}
if (LOG.isDebugEnabled())
LOG.debug("isReady(), no content for needContent retry {}", this);
return false;
@ -295,7 +312,7 @@ class AsyncContentProducer implements ContentProducer
return _httpChannel.getState().isInputUnready();
}
private Content.Chunk nextTransformedContent()
private HttpInput.Content nextTransformedContent()
{
if (LOG.isDebugEnabled())
LOG.debug("nextTransformedContent {}", this);
@ -304,19 +321,18 @@ class AsyncContentProducer implements ContentProducer
{
if (_transformedContent != null)
{
if (_transformedContent.isTerminal() || _transformedContent.hasRemaining())
if (_transformedContent.isSpecial() || !_transformedContent.isEmpty())
{
if (_transformedContent instanceof Content.Chunk.Error && !_error)
if (_transformedContent.getError() != null && !_error)
{
// In case the _rawContent was set by consumeAll(), check the httpChannel
// to see if it has a more precise error. Otherwise, the exact same
// special content will be returned by the httpChannel; do not do that
// if the _error flag was set, meaning the current error is definitive.
Content.Chunk refreshedRawContent = produceRawContent();
HttpInput.Content refreshedRawContent = produceRawContent();
if (refreshedRawContent != null)
_rawContent = _transformedContent = refreshedRawContent;
_error = _rawContent instanceof Content.Chunk.Error;
_error = _rawContent.getError() != null;
if (LOG.isDebugEnabled())
LOG.debug("refreshed raw content: {} {}", _rawContent, this);
}
@ -329,8 +345,7 @@ class AsyncContentProducer implements ContentProducer
{
if (LOG.isDebugEnabled())
LOG.debug("current transformed content depleted {}", this);
_transformedContent.release();
_transformedContent.succeeded();
_transformedContent = null;
}
}
@ -363,42 +378,42 @@ class AsyncContentProducer implements ContentProducer
_transformedContent = intercept();
// If the interceptor generated a special content, _rawContent must become that special content.
if (_transformedContent != null && _transformedContent.isTerminal() && _transformedContent != _rawContent)
if (_transformedContent != null && _transformedContent.isSpecial() && _transformedContent != _rawContent)
{
if (LOG.isDebugEnabled())
LOG.debug("interceptor generated a special content, _rawContent must become that special content {}", this);
_rawContent.release();
_rawContent.succeeded();
_rawContent = _transformedContent;
return;
}
// If the interceptor generated a null content, recycle the raw content now if it is empty.
if (_transformedContent == null && !_rawContent.hasRemaining() && !_rawContent.isTerminal())
if (_transformedContent == null && _rawContent.isEmpty() && !_rawContent.isSpecial())
{
if (LOG.isDebugEnabled())
LOG.debug("interceptor generated a null content, recycling the empty raw content now {}", this);
_rawContent.release();
_rawContent.succeeded();
_rawContent = null;
return;
}
// If the interceptor returned the raw content, recycle the raw content now if it is empty.
if (_transformedContent == _rawContent && !_rawContent.hasRemaining() && !_rawContent.isTerminal())
if (_transformedContent == _rawContent && _rawContent.isEmpty() && !_rawContent.isSpecial())
{
if (LOG.isDebugEnabled())
LOG.debug("interceptor returned the raw content, recycle the empty raw content now {}", this);
_rawContent.release();
_rawContent.succeeded();
_rawContent = _transformedContent = null;
}
}
else
{
// Recycle the raw content now if it is empty.
if (!_rawContent.hasRemaining() && !_rawContent.isTerminal())
if (_rawContent.isEmpty() && !_rawContent.isSpecial())
{
if (LOG.isDebugEnabled())
LOG.debug("recycling the empty raw content now {}", this);
_rawContent.release();
_rawContent.succeeded();
_rawContent = null;
}
@ -408,39 +423,39 @@ class AsyncContentProducer implements ContentProducer
}
}
private Content.Chunk intercept()
private HttpInput.Content intercept()
{
try
{
int remainingBeforeInterception = _rawContent.remaining();
Content.Chunk content = _interceptor.readFrom(_rawContent);
if (content != null && content.isTerminal() && !_rawContent.isTerminal())
HttpInput.Content content = _interceptor.readFrom(_rawContent);
if (content != null && content.isSpecial() && !_rawContent.isSpecial())
{
if (content instanceof Content.Chunk.Error errorContent)
Throwable error = content.getError();
if (error != null)
{
// Set the _error flag to mark the content as definitive, i.e.:
// do not try to produce new raw content to get a fresher error
// when the special content was generated by the interceptor.
_error = true;
if (_httpChannel.getResponse().isCommitted())
_httpChannel.abort(errorContent.getCause());
_httpChannel.abort(error);
}
if (LOG.isDebugEnabled())
LOG.debug("interceptor generated special content {}", this);
}
else if (content != _rawContent && !_rawContent.isTerminal() && _rawContent.hasRemaining() && _rawContent.remaining() == remainingBeforeInterception)
else if (content != _rawContent && !_rawContent.isSpecial() && !_rawContent.isEmpty() && _rawContent.remaining() == remainingBeforeInterception)
{
IOException failure = new IOException("Interceptor " + _interceptor + " did not consume any of the " + _rawContent.remaining() + " remaining byte(s) of content");
if (content != null)
content.release();
releaseContent();
content.failed(failure);
failCurrentContent(failure);
// Set the _error flag to mark the content as definitive, i.e.:
// do not try to produce new raw content to get a fresher error
// when the special content was caused by the interceptor not
// consuming the raw content.
_error = true;
org.eclipse.jetty.server.Response response = _httpChannel.getCoreResponse();
Response response = _httpChannel.getResponse();
if (response.isCommitted())
_httpChannel.abort(failure);
if (LOG.isDebugEnabled())
@ -455,12 +470,12 @@ class AsyncContentProducer implements ContentProducer
catch (Throwable x)
{
IOException failure = new IOException("Bad content", x);
releaseContent();
failCurrentContent(failure);
// Set the _error flag to mark the content as definitive, i.e.:
// do not try to produce new raw content to get a fresher error
// when the special content was caused by the interceptor throwing.
_error = true;
Response response = _httpChannel.getCoreResponse();
Response response = _httpChannel.getResponse();
if (response.isCommitted())
_httpChannel.abort(failure);
if (LOG.isDebugEnabled())
@ -469,9 +484,9 @@ class AsyncContentProducer implements ContentProducer
}
}
private Content.Chunk produceRawContent()
private HttpInput.Content produceRawContent()
{
Content.Chunk content = _httpChannel.getCoreRequest().read();
HttpInput.Content content = _httpChannel.produceContent();
if (content != null)
{
_rawContentArrived += content.remaining();
@ -479,14 +494,6 @@ class AsyncContentProducer implements ContentProducer
_firstByteNanoTime = NanoTime.now();
if (LOG.isDebugEnabled())
LOG.debug("produceRawContent updated rawContentArrived to {} and firstByteTimeStamp to {} {}", _rawContentArrived, _firstByteNanoTime, this);
// TODO do we really need these listeners?
if (content.hasRemaining())
_httpChannel.onContent(content);
if (content instanceof Trailers trailers)
_httpChannel.onTrailers(trailers.getTrailers());
if (content.isLast())
_httpChannel.onContentComplete();
}
if (LOG.isDebugEnabled())
LOG.debug("produceRawContent produced {} {}", content, this);
@ -502,13 +509,14 @@ class AsyncContentProducer implements ContentProducer
@Override
public String toString()
{
return String.format("%s@%x[r=%s,t=%s,i=%s,error=%b]",
return String.format("%s@%x[r=%s,t=%s,i=%s,error=%b,c=%s]",
getClass().getSimpleName(),
hashCode(),
_rawContent,
_transformedContent,
_interceptor,
_error
_error,
_httpChannel
);
}

View File

@ -13,7 +13,6 @@
package org.eclipse.jetty.ee9.nested;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.util.thread.AutoLock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -97,19 +96,11 @@ class BlockingContentProducer implements ContentProducer
}
@Override
public boolean releaseContent()
{
boolean eof = _asyncContentProducer.releaseContent();
_semaphore.release();
return eof;
}
@Override
public Content.Chunk nextContent()
public HttpInput.Content nextContent()
{
while (true)
{
Content.Chunk content = _asyncContentProducer.nextContent();
HttpInput.Content content = _asyncContentProducer.nextContent();
if (LOG.isDebugEnabled())
LOG.debug("nextContent async producer returned {}", content);
if (content != null)
@ -132,13 +123,13 @@ class BlockingContentProducer implements ContentProducer
}
catch (InterruptedException e)
{
return Content.Chunk.from(e);
return new HttpInput.ErrorContent(e);
}
}
}
@Override
public void reclaim(Content.Chunk content)
public void reclaim(HttpInput.Content content)
{
_asyncContentProducer.reclaim(content);
}

View File

@ -14,7 +14,6 @@
package org.eclipse.jetty.ee9.nested;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.server.HttpChannel;
import org.eclipse.jetty.util.component.Destroyable;
import org.eclipse.jetty.util.thread.AutoLock;
@ -42,23 +41,15 @@ public interface ContentProducer
*/
void reopen();
/**
* Consume all content currently available in this {@link ContentProducer} instance
* as well as in the underlying {@link HttpChannel}.
* Consumes all content currently available in this {@link ContentProducer} instance
* as well as in the underlying {@link Content.Source}.
*
* This call is always non-blocking.
* @return true if EOF was reached.
*/
boolean consumeAll();
/**
* Release any held content.
*
* @return true if EOF seen.
*/
boolean releaseContent();
/**
* Check if the current data rate consumption is above the minimal rate.
* Abort the channel, fail the content currently available and throw a
@ -116,13 +107,13 @@ public interface ContentProducer
* @return the next content that can be read from or null if the implementation does not block
* and has no available content.
*/
Content.Chunk nextContent();
HttpInput.Content nextContent();
/**
* Free up the content by calling {@link Content.Chunk#release()} on it
* Free up the content by calling {@link HttpInput.Content#succeeded()} on it
* and updating this instance' internal state.
*/
void reclaim(Content.Chunk content);
void reclaim(HttpInput.Content content);
/**
* Check if this {@link ContentProducer} instance has some content that can be read without blocking.

View File

@ -42,6 +42,7 @@ import org.eclipse.jetty.http.HttpHeaderValue;
import org.eclipse.jetty.http.HttpStatus;
import org.eclipse.jetty.http.HttpVersion;
import org.eclipse.jetty.http.MetaData;
import org.eclipse.jetty.http.Trailers;
import org.eclipse.jetty.io.ByteBufferPool;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.Content;
@ -139,6 +140,72 @@ public class HttpChannel implements Runnable, HttpOutput.Interceptor
return _connectionMetaData;
}
/**
* Notify the channel that content is needed. If some content is immediately available, true is returned and
* {@link #produceContent()} has to be called and will return a non-null object.
* If no content is immediately available, an attempt to produce content must be made; if new content has been
* produced, true is returned; otherwise {@link HttpInput#onContentProducible()} is called once some content
* arrives and {@link #produceContent()} can be called without returning {@code null}.
* If a failure happens, then {@link HttpInput#onContentProducible()} will be called and an error content will
* return the error on the next call to {@link #produceContent()}.
* @return true if content is immediately available.
*/
public boolean needContent()
{
// TODO: optimize by attempting a read?
getCoreRequest().demand(() ->
{
if (getRequest().getHttpInput().onContentProducible())
handle();
});
return false;
}
/**
* Produce a {@link HttpInput.Content} object with data currently stored within the channel. The produced content
* can be special (meaning calling {@link HttpInput.Content#isSpecial()} returns true) if the channel reached a special
* state, like EOF or an error.
* Once a special content has been returned, all subsequent calls to this method will always return a special content
* of the same kind and {@link #needContent()} will always return true.
* The returned content is "raw", i.e.: not decoded.
* @return a {@link HttpInput.Content} object if one is immediately available without blocking, null otherwise.
*/
public HttpInput.Content produceContent()
{
Content.Chunk chunk = getCoreRequest().read();
if (chunk == null)
return null;
if (chunk.hasRemaining())
onContent(chunk);
if (chunk instanceof Trailers trailers)
onTrailers(trailers.getTrailers());
if (chunk.isLast())
onContentComplete();
return HttpInput.Content.asChunk(chunk);
}
/**
* Fail all content that is currently stored within the channel.
* @param failure the failure to fail the content with.
* @return true if EOF was reached while failing all content, false otherwise.
*/
public boolean failAllContent(Throwable failure)
{
while (true)
{
HttpInput.Content content = produceContent();
if (content == null)
return false;
if (content.isSpecial())
return content.isEof();
content.failed(failure);
if (content.isEof())
return true;
}
}
/**
* Fail the channel's input.
* @param failure the failure.
@ -898,7 +965,7 @@ public class HttpChannel implements Runnable, HttpOutput.Interceptor
{
if (LOG.isDebugEnabled())
LOG.debug("onContent {} {}", this, content);
_combinedListener.onRequestContent(_request, content.getByteBuffer());
_combinedListener.onRequestContent(_request, content.getByteBuffer().slice());
}
void onContentComplete()

View File

@ -21,8 +21,8 @@ import java.util.concurrent.atomic.LongAdder;
import jakarta.servlet.ReadListener;
import jakarta.servlet.ServletInputStream;
import org.eclipse.jetty.http.HttpFields;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.server.Context;
import org.eclipse.jetty.util.BufferUtil;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.component.Destroyable;
import org.eclipse.jetty.util.thread.AutoLock;
@ -37,10 +37,10 @@ public class HttpInput extends ServletInputStream implements Runnable
{
private static final Logger LOG = LoggerFactory.getLogger(HttpInput.class);
private final HttpChannel _httpChannel;
private final byte[] _oneByteBuffer = new byte[1];
private BlockingContentProducer _blockingContentProducer;
private AsyncContentProducer _asyncContentProducer;
private final BlockingContentProducer _blockingContentProducer;
private final AsyncContentProducer _asyncContentProducer;
private final HttpChannel _httpChannel;
private final LongAdder _contentConsumed = new LongAdder();
private volatile ContentProducer _contentProducer;
private volatile boolean _consumedEof;
@ -131,18 +131,18 @@ public class HttpInput extends ServletInputStream implements Runnable
}
}
private int get(Content.Chunk chunk, byte[] bytes, int offset, int length)
private int get(Content content, byte[] bytes, int offset, int length)
{
length = Math.min(chunk.remaining(), length);
chunk.getByteBuffer().get(bytes, offset, length);
_contentConsumed.add(length);
return length;
length = Math.min(content.remaining(), length);
int consumed = content.get(bytes, offset, length);
_contentConsumed.add(consumed);
return consumed;
}
private int get(Content.Chunk chunk, ByteBuffer des)
private int get(Content content, ByteBuffer des)
{
var capacity = des.remaining();
var src = chunk.getByteBuffer();
var src = content.getByteBuffer();
if (src.remaining() > capacity)
{
int limit = src.limit();
@ -172,18 +172,6 @@ public class HttpInput extends ServletInputStream implements Runnable
}
}
public void releaseContent()
{
try (AutoLock lock = _contentProducer.lock())
{
if (LOG.isDebugEnabled())
LOG.debug("consumeAll {}", this);
boolean eof = _contentProducer.releaseContent();
if (eof)
_consumedEof = true;
}
}
public boolean consumeAll()
{
try (AutoLock lock = _contentProducer.lock())
@ -302,22 +290,32 @@ public class HttpInput extends ServletInputStream implements Runnable
// Calculate minimum request rate for DoS protection
_contentProducer.checkMinDataRate();
Content.Chunk content = _contentProducer.nextContent();
Content content = _contentProducer.nextContent();
if (content == null)
throw new IllegalStateException("read on unready input");
if (content.isTerminal())
if (!content.isSpecial())
{
if (content instanceof Content.Chunk.Error errorContent)
{
Throwable error = errorContent.getCause();
if (LOG.isDebugEnabled())
LOG.debug("read error={} {}", error, this);
if (error instanceof IOException)
throw (IOException)error;
throw new IOException(error);
}
int read = buffer == null ? get(content, b, off, len) : get(content, buffer);
if (LOG.isDebugEnabled())
LOG.debug("read produced {} byte(s) {}", read, this);
if (content.isEmpty())
_contentProducer.reclaim(content);
return read;
}
Throwable error = content.getError();
if (LOG.isDebugEnabled())
LOG.debug("read error={} {}", error, this);
if (error != null)
{
if (error instanceof IOException)
throw (IOException)error;
throw new IOException(error);
}
if (content.isEof())
{
if (LOG.isDebugEnabled())
LOG.debug("read at EOF, setting consumed EOF to true {}", this);
_consumedEof = true;
@ -326,15 +324,8 @@ public class HttpInput extends ServletInputStream implements Runnable
scheduleReadListenerNotification();
return -1;
}
else
{
int read = buffer == null ? get(content, b, off, len) : get(content, buffer);
if (LOG.isDebugEnabled())
LOG.debug("read produced {} byte(s) {}", read, this);
if (!content.hasRemaining())
_contentProducer.reclaim(content);
return read;
}
throw new AssertionError("no data, no error and not EOF");
}
}
@ -382,7 +373,7 @@ public class HttpInput extends ServletInputStream implements Runnable
@Override
public void run()
{
Content.Chunk content;
Content content;
try (AutoLock lock = _contentProducer.lock())
{
// Call isReady() to make sure that if not ready we register for fill interest.
@ -406,18 +397,18 @@ public class HttpInput extends ServletInputStream implements Runnable
return;
}
if (content.isTerminal())
if (content.isSpecial())
{
if (content instanceof Content.Chunk.Error errorContent)
Throwable error = content.getError();
if (error != null)
{
Throwable error = errorContent.getCause();
if (LOG.isDebugEnabled())
LOG.debug("running error={} {}", error, this);
// TODO is this necessary to add here?
_httpChannel.getResponse().getHttpFields().add(HttpFields.CONNECTION_CLOSE);
_readListener.onError(error);
}
else
else if (content.isEof())
{
try
{
@ -460,9 +451,9 @@ public class HttpInput extends ServletInputStream implements Runnable
}
/**
* <p>{@link Content.Chunk} interceptor that can be registered using {@link #setInterceptor(Interceptor)} or
* <p>{@link Content} interceptor that can be registered using {@link #setInterceptor(Interceptor)} or
* {@link #addInterceptor(Interceptor)}.
* When {@link Content.Chunk} instances are generated, they are passed to the registered interceptor (if any)
* When {@link Content} instances are generated, they are passed to the registered interceptor (if any)
* that is then responsible for providing the actual content that is consumed by {@link #read(byte[], int, int)} and its
* sibling methods.</p>
* A minimal implementation could be as simple as:
@ -487,21 +478,24 @@ public class HttpInput extends ServletInputStream implements Runnable
* </pre>
* Implementors of this interface must keep the following in mind:
* <ul>
* <li>Calling {@link Content.Chunk#getByteBuffer()} when {@link Content.Chunk#isTerminal()} returns <code>true</code> throws
* <li>Calling {@link Content#getByteBuffer()} when {@link Content#isSpecial()} returns <code>true</code> throws
* {@link IllegalStateException}.</li>
* <li>A {@link Content.Chunk} can both be non-special and have {@code content == Content.EOF} return <code>true</code>.</li>
* <li>{@link Content.Chunk} extends {@link Callback} to manage the lifecycle of the contained byte buffer. The code calling
* {@link #readFrom(Content.Chunk)} is responsible for managing the lifecycle of both the passed and the returned content
* instances, once {@link ByteBuffer#hasRemaining()} returns <code>false</code> {@code HttpInput} will make sure
* <li>A {@link Content} can both be non-special and have {@link Content#isEof()} return <code>true</code>.</li>
* <li>{@link Content} extends {@link Callback} to manage the lifecycle of the contained byte buffer. The code calling
* {@link #readFrom(Content)} is responsible for managing the lifecycle of both the passed and the returned content
* instances, once {@link ByteBuffer#hasRemaining()} returns <code>false</code> {@link HttpInput} will make sure
* {@link Callback#succeeded()} is called, or {@link Callback#failed(Throwable)} if an error occurs.</li>
* <li>After {@link #readFrom(Content.Chunk)} is called for the first time, subsequent {@link #readFrom(Content.Chunk)} calls will
* <li>After {@link #readFrom(Content)} is called for the first time, subsequent {@link #readFrom(Content)} calls will
* occur only after the contained byte buffer is empty (see above) or at any time if the returned content was special.</li>
* <li>Once {@link #readFrom(Content.Chunk)} returned a special content, subsequent calls to {@link #readFrom(Content.Chunk)} must
* <li>Once {@link #readFrom(Content)} returned a special content, subsequent calls to {@link #readFrom(Content)} must
* always return the same special content.</li>
* <li>Implementations implementing both this interface and {@link Destroyable} will have their
* {@link Destroyable#destroy()} method called when {@link #recycle()} is called.</li>
* </ul>
*
* @deprecated Interceptor has been removed with no replacement in the EE10 implementation
*/
@Deprecated(forRemoval = true)
public interface Interceptor
{
/**
@ -511,13 +505,13 @@ public class HttpInput extends ServletInputStream implements Runnable
* unless the returned content is the passed content instance.
* @return The intercepted content or null if interception is completed for that content.
*/
Content.Chunk readFrom(Content.Chunk content);
Content readFrom(Content content);
}
/**
* An {@link Interceptor} that chains two other {@link Interceptor}s together.
* The {@link Interceptor#readFrom(Content.Chunk)} calls the previous {@link Interceptor}'s
* {@link Interceptor#readFrom(Content.Chunk)} and then passes any {@link Content.Chunk} returned
* The {@link Interceptor#readFrom(Content)} calls the previous {@link Interceptor}'s
* {@link Interceptor#readFrom(Content)} and then passes any {@link Content} returned
* to the next {@link Interceptor}.
*/
private static class ChainedInterceptor implements Interceptor, Destroyable
@ -542,9 +536,9 @@ public class HttpInput extends ServletInputStream implements Runnable
}
@Override
public Content.Chunk readFrom(Content.Chunk content)
public Content readFrom(Content content)
{
Content.Chunk c = getPrev().readFrom(content);
Content c = getPrev().readFrom(content);
if (c == null)
return null;
return getNext().readFrom(c);
@ -565,4 +559,293 @@ public class HttpInput extends ServletInputStream implements Runnable
return getClass().getSimpleName() + "@" + hashCode() + " [p=" + _prev + ",n=" + _next + "]";
}
}
/**
* A content represents the production of context bytes by {@link AsyncContentProducer#nextContent()}.
* There are two fundamental types of content: special and non-special.
* Non-special content always wraps a byte buffer that can be consumed and must be recycled once it is empty, either
* via {@link #succeeded()} or {@link #failed(Throwable)}.
* Special content indicates a special event, like EOF or an error and never wraps a byte buffer. Calling
* {@link #succeeded()} or {@link #failed(Throwable)} on those have no effect.
*/
public static class Content implements Callback
{
public static Content asChunk(org.eclipse.jetty.io.Content.Chunk chunk)
{
if (chunk instanceof org.eclipse.jetty.io.Content.Chunk.Error error)
return new ErrorContent(error.getCause());
if (chunk.isLast() && !chunk.hasRemaining())
return new EofContent();
Content content = new Content(chunk.getByteBuffer())
{
@Override
public void succeeded()
{
chunk.release();
super.succeeded();
}
@Override
public void failed(Throwable x)
{
chunk.release();
super.failed(x);
}
};
return chunk.isLast() ? new WrappingContent(content, true) : content;
}
protected final ByteBuffer _content;
public Content(ByteBuffer content)
{
_content = content;
}
/**
* Get the wrapped byte buffer. Throws {@link IllegalStateException} if the content is special.
* @return the wrapped byte buffer.
*/
public ByteBuffer getByteBuffer()
{
return _content;
}
@Override
public InvocationType getInvocationType()
{
return InvocationType.NON_BLOCKING;
}
/**
* Read the wrapped byte buffer. Throws {@link IllegalStateException} if the content is special.
* @param buffer The array into which bytes are to be written.
* @param offset The offset within the array of the first byte to be written.
* @param length The maximum number of bytes to be written to the given array.
* @return The amount of bytes read from the buffer.
*/
public int get(byte[] buffer, int offset, int length)
{
length = Math.min(_content.remaining(), length);
_content.get(buffer, offset, length);
return length;
}
/**
* Skip some bytes from the buffer. Has no effect on a special content.
* @param length How many bytes to skip.
* @return How many bytes were skipped.
*/
public int skip(int length)
{
length = Math.min(_content.remaining(), length);
_content.position(_content.position() + length);
return length;
}
/**
* Check if there is at least one byte left in the buffer.
* Always false on a special content.
* @return true if there is at least one byte left in the buffer.
*/
public boolean hasContent()
{
return _content.hasRemaining();
}
/**
* Get the number of bytes remaining in the buffer.
* Always 0 on a special content.
* @return the number of bytes remaining in the buffer.
*/
public int remaining()
{
return _content.remaining();
}
/**
* Check if the buffer is empty.
* Always true on a special content.
* @return true if there is 0 byte left in the buffer.
*/
public boolean isEmpty()
{
return !_content.hasRemaining();
}
/**
* Check if the content is special. A content is deemed special
* if it does not hold bytes but rather conveys a special event,
* like when EOF has been reached or an error has occurred.
* @return true if the content is special, false otherwise.
*/
public boolean isSpecial()
{
return false;
}
/**
* Check if EOF was reached. Both special and non-special content
* can have this flag set to true but in the case of non-special content,
* this can be interpreted as a hint as it is always going to be followed
* by another content that is both special and EOF.
* @return true if EOF was reached, false otherwise.
*/
public boolean isEof()
{
return false;
}
/**
* Get the reported error. Only special contents can have an error.
* @return the error or null if there is none.
*/
public Throwable getError()
{
return null;
}
@Override
public String toString()
{
return String.format("%s@%x{%s,spc=%s,eof=%s,err=%s}", getClass().getSimpleName(), hashCode(),
BufferUtil.toDetailString(_content), isSpecial(), isEof(), getError());
}
}
/**
* Simple non-special content wrapper allow overriding the EOF flag.
*/
public static class WrappingContent extends Content
{
private final Content _delegate;
private final boolean _eof;
public WrappingContent(Content delegate, boolean eof)
{
super(delegate.getByteBuffer());
_delegate = delegate;
_eof = eof;
}
@Override
public boolean isEof()
{
return _eof;
}
@Override
public void succeeded()
{
_delegate.succeeded();
}
@Override
public void failed(Throwable x)
{
_delegate.failed(x);
}
@Override
public InvocationType getInvocationType()
{
return _delegate.getInvocationType();
}
}
/**
* Abstract class that implements the standard special content behavior.
*/
public abstract static class SpecialContent extends Content
{
public SpecialContent()
{
super(null);
}
@Override
public final ByteBuffer getByteBuffer()
{
throw new IllegalStateException(this + " has no buffer");
}
@Override
public final int get(byte[] buffer, int offset, int length)
{
throw new IllegalStateException(this + " has no buffer");
}
@Override
public final int skip(int length)
{
return 0;
}
@Override
public final boolean hasContent()
{
return false;
}
@Override
public final int remaining()
{
return 0;
}
@Override
public final boolean isEmpty()
{
return true;
}
@Override
public final boolean isSpecial()
{
return true;
}
}
/**
* EOF special content.
*/
public static final class EofContent extends SpecialContent
{
@Override
public boolean isEof()
{
return true;
}
@Override
public String toString()
{
return getClass().getSimpleName();
}
}
/**
* Error special content.
*/
public static final class ErrorContent extends SpecialContent
{
private final Throwable _error;
public ErrorContent(Throwable error)
{
_error = error;
}
@Override
public Throwable getError()
{
return _error;
}
@Override
public String toString()
{
return getClass().getSimpleName() + " [" + _error + "]";
}
}
}

View File

@ -1305,9 +1305,10 @@ public class Request implements HttpServletRequest
*/
public void onCompleted()
{
_input.releaseContent();
// Clean up unread content.
_input.consumeAll();
//Clean up any tmp files created by MultiPartInputStream
// Clean up any tmp files created by MultiPartInputStream.
if (_multiParts != null)
{
try

View File

@ -21,7 +21,6 @@ import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
@ -30,15 +29,14 @@ import java.util.zip.GZIPOutputStream;
import jakarta.servlet.ServletException;
import jakarta.servlet.http.HttpServletRequest;
import jakarta.servlet.http.HttpServletResponse;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EofException;
import org.eclipse.jetty.server.LocalConnector;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.util.component.Destroyable;
import org.eclipse.jetty.util.thread.AutoLock;
import org.eclipse.jetty.util.thread.Scheduler;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import static org.hamcrest.MatcherAssert.assertThat;
@ -47,7 +45,6 @@ import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
@Disabled // TODO
public class AsyncContentProducerTest
{
private Server _server;
@ -65,6 +62,7 @@ public class AsyncContentProducerTest
_server.setHandler(_contextHandler);
_testHandler = new TestHandler();
_contextHandler.setHandler(_testHandler);
_server.start();
}
@AfterEach
@ -101,7 +99,7 @@ public class AsyncContentProducerTest
CyclicBarrier barrier = new CyclicBarrier(2);
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, Content.Chunk.EOF, null, barrier));
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, new HttpInput.EofContent(), _connector.getScheduler(), barrier));
try (AutoLock ignored = contentProducer.lock())
{
@ -123,7 +121,7 @@ public class AsyncContentProducerTest
CyclicBarrier barrier = new CyclicBarrier(2);
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, Content.Chunk.from(expectedError), null, barrier));
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, new HttpInput.ErrorContent(expectedError), _connector.getScheduler(), barrier));
try (AutoLock ignored = contentProducer.lock())
{
@ -144,7 +142,7 @@ public class AsyncContentProducerTest
CyclicBarrier barrier = new CyclicBarrier(2);
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, Content.Chunk.EOF, null, barrier));
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, new HttpInput.EofContent(), _connector.getScheduler(), barrier));
AccountingInterceptor interceptor = new AccountingInterceptor();
try (AutoLock ignored = contentProducer.lock())
{
@ -153,17 +151,17 @@ public class AsyncContentProducerTest
Throwable error = readAndAssertContent(totalContentBytesCount, originalContentString, contentProducer, (buffers.length + 1) * 2, 0, 4, barrier);
assertThat(error, nullValue());
Content.Chunk lastContent = contentProducer.nextContent();
assertThat(lastContent.isTerminal(), is(true));
assertThat(lastContent.isLast(), is(true));
HttpInput.Content lastContent = contentProducer.nextContent();
assertThat(lastContent.isSpecial(), is(true));
assertThat(lastContent.isEof(), is(true));
}
assertThat(interceptor.contents.size(), is(4));
assertThat(interceptor.contents.get(0).isTerminal(), is(false));
assertThat(interceptor.contents.get(1).isTerminal(), is(false));
assertThat(interceptor.contents.get(2).isTerminal(), is(false));
assertThat(interceptor.contents.get(3).isTerminal(), is(true));
assertThat(interceptor.contents.get(3).isLast(), is(true));
assertThat(interceptor.contents.get(0).isSpecial(), is(false));
assertThat(interceptor.contents.get(1).isSpecial(), is(false));
assertThat(interceptor.contents.get(2).isSpecial(), is(false));
assertThat(interceptor.contents.get(3).isSpecial(), is(true));
assertThat(interceptor.contents.get(3).isEof(), is(true));
}
@Test
@ -178,7 +176,7 @@ public class AsyncContentProducerTest
CyclicBarrier barrier = new CyclicBarrier(2);
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, Content.Chunk.from(new Throwable("testAsyncContentProducerErrorContentIsPassedToInterceptor error")), null, barrier));
ContentProducer contentProducer = new AsyncContentProducer(new ArrayDelayedHttpChannel(buffers, new HttpInput.ErrorContent(new Throwable("testAsyncContentProducerErrorContentIsPassedToInterceptor error")), _connector.getScheduler(), barrier));
AccountingInterceptor interceptor = new AccountingInterceptor();
try (AutoLock ignored = contentProducer.lock())
{
@ -187,70 +185,91 @@ public class AsyncContentProducerTest
Throwable error = readAndAssertContent(totalContentBytesCount, originalContentString, contentProducer, (buffers.length + 1) * 2, 0, 4, barrier);
assertThat(error.getMessage(), is("testAsyncContentProducerErrorContentIsPassedToInterceptor error"));
Content.Chunk lastContent = contentProducer.nextContent();
assertThat(lastContent.isTerminal(), is(true));
assertThat(getError(lastContent).getMessage(), is("testAsyncContentProducerErrorContentIsPassedToInterceptor error"));
HttpInput.Content lastContent = contentProducer.nextContent();
assertThat(lastContent.isSpecial(), is(true));
assertThat(lastContent.getError().getMessage(), is("testAsyncContentProducerErrorContentIsPassedToInterceptor error"));
}
assertThat(interceptor.contents.size(), is(4));
assertThat(interceptor.contents.get(0).isTerminal(), is(false));
assertThat(interceptor.contents.get(1).isTerminal(), is(false));
assertThat(interceptor.contents.get(2).isTerminal(), is(false));
assertThat(interceptor.contents.get(3).isTerminal(), is(true));
assertThat(getError(interceptor.contents.get(3)).getMessage(), is("testAsyncContentProducerErrorContentIsPassedToInterceptor error"));
assertThat(interceptor.contents.get(0).isSpecial(), is(false));
assertThat(interceptor.contents.get(1).isSpecial(), is(false));
assertThat(interceptor.contents.get(2).isSpecial(), is(false));
assertThat(interceptor.contents.get(3).isSpecial(), is(true));
assertThat(interceptor.contents.get(3).getError().getMessage(), is("testAsyncContentProducerErrorContentIsPassedToInterceptor error"));
}
@Test
public void testAsyncContentProducerInterceptorGeneratesError()
{
AtomicInteger contentReleasedCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(Content.Chunk.from(ByteBuffer.allocate(1), false)), Content.Chunk.EOF));
AtomicInteger contentSucceededCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void succeeded()
{
contentSucceededCount.incrementAndGet();
}
}), new HttpInput.EofContent()));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content -> Content.Chunk.from(new Throwable("testAsyncContentProducerInterceptorGeneratesError interceptor error")));
contentProducer.setInterceptor(content -> new HttpInput.ErrorContent(new Throwable("testAsyncContentProducerInterceptorGeneratesError interceptor error")));
assertThat(contentProducer.isReady(), is(true));
assertThat(contentProducer.isError(), is(true));
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(getError(content1).getMessage(), is("testAsyncContentProducerInterceptorGeneratesError interceptor error"));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.getError().getMessage(), is("testAsyncContentProducerInterceptorGeneratesError interceptor error"));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(getError(content2).getMessage(), is("testAsyncContentProducerInterceptorGeneratesError interceptor error"));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.getError().getMessage(), is("testAsyncContentProducerInterceptorGeneratesError interceptor error"));
}
assertThat(contentReleasedCount.get(), is(1));
assertThat(contentSucceededCount.get(), is(1));
}
@Test
public void testAsyncContentProducerInterceptorGeneratesEof()
{
AtomicInteger contentReleasedCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(Content.Chunk.from(ByteBuffer.allocate(1), false)), Content.Chunk.from(new Throwable("should not reach this"))));
AtomicInteger contentSucceededCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void succeeded()
{
contentSucceededCount.incrementAndGet();
}
}), new HttpInput.ErrorContent(new Throwable("should not reach this"))));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content -> Content.Chunk.EOF);
contentProducer.setInterceptor(content -> new HttpInput.EofContent());
assertThat(contentProducer.isReady(), is(true));
assertThat(contentProducer.isError(), is(false));
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(content1.isLast(), is(true));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.isEof(), is(true));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(content2.isLast(), is(true));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.isEof(), is(true));
}
assertThat(contentReleasedCount.get(), is(1));
assertThat(contentSucceededCount.get(), is(1));
}
@Test
public void testAsyncContentProducerInterceptorThrows()
{
AtomicInteger contentReleasedCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(Content.Chunk.from(ByteBuffer.allocate(1), false)), Content.Chunk.EOF));
AtomicInteger contentFailedCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void failed(Throwable x)
{
contentFailedCount.incrementAndGet();
}
}), new HttpInput.EofContent()));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content ->
@ -261,53 +280,74 @@ public class AsyncContentProducerTest
assertThat(contentProducer.isReady(), is(true));
assertThat(contentProducer.isError(), is(true));
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(getError(content1).getCause().getMessage(), is("testAsyncContentProducerInterceptorThrows error"));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.getError().getCause().getMessage(), is("testAsyncContentProducerInterceptorThrows error"));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(getError(content2).getCause().getMessage(), is("testAsyncContentProducerInterceptorThrows error"));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.getError().getCause().getMessage(), is("testAsyncContentProducerInterceptorThrows error"));
}
assertThat(contentReleasedCount.get(), is(1));
assertThat(contentFailedCount.get(), is(1));
}
@Test
public void testAsyncContentProducerInterceptorDoesNotConsume()
{
AtomicInteger contentReleasedCount = new AtomicInteger();
AtomicInteger interceptorContentReleasedCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(Content.Chunk.from(ByteBuffer.allocate(1), false)), Content.Chunk.EOF));
AtomicInteger contentFailedCount = new AtomicInteger();
AtomicInteger interceptorContentFailedCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void failed(Throwable x)
{
contentFailedCount.incrementAndGet();
}
}), new HttpInput.EofContent()));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content -> Content.Chunk.from(ByteBuffer.allocate(1), false));
contentProducer.setInterceptor(content -> new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void failed(Throwable x)
{
interceptorContentFailedCount.incrementAndGet();
}
});
assertThat(contentProducer.isReady(), is(true));
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(getError(content1).getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.getError().getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(getError(content2).getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.getError().getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
}
assertThat(contentReleasedCount.get(), is(1));
assertThat(interceptorContentReleasedCount.get(), is(1));
assertThat(contentFailedCount.get(), is(1));
assertThat(interceptorContentFailedCount.get(), is(1));
}
@Test
public void testAsyncContentProducerInterceptorDoesNotConsumeEmptyContent()
{
AtomicInteger contentReleasedCount = new AtomicInteger();
AtomicInteger contentSucceededCount = new AtomicInteger();
AtomicInteger specialContentInterceptedCount = new AtomicInteger();
AtomicInteger nullContentInterceptedCount = new AtomicInteger();
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(Content.Chunk.from(ByteBuffer.allocate(0), false)), Content.Chunk.EOF));
ContentProducer contentProducer = new AsyncContentProducer(new ContentListHttpChannel(List.of(new HttpInput.Content(ByteBuffer.allocate(0))
{
@Override
public void succeeded()
{
contentSucceededCount.incrementAndGet();
}
}), new HttpInput.EofContent()));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content ->
{
if (content.isTerminal())
if (content.isSpecial())
{
specialContentInterceptedCount.incrementAndGet();
return content;
@ -316,15 +356,15 @@ public class AsyncContentProducerTest
return null;
});
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(content1.isLast(), is(true));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.isEof(), is(true));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(content2.isLast(), is(true));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.isEof(), is(true));
}
assertThat(contentReleasedCount.get(), is(1));
assertThat(contentSucceededCount.get(), is(1));
assertThat(specialContentInterceptedCount.get(), is(1));
assertThat(nullContentInterceptedCount.get(), is(1));
}
@ -345,7 +385,7 @@ public class AsyncContentProducerTest
else
isReadyFalseCount++;
Content.Chunk content = contentProducer.nextContent();
HttpInput.Content content = contentProducer.nextContent();
nextContentCount++;
if (content == null)
{
@ -355,11 +395,11 @@ public class AsyncContentProducerTest
}
assertThat(content, notNullValue());
if (content.isTerminal())
if (content.isSpecial())
{
if (content.isLast())
if (content.isEof())
break;
error = getError(content);
error = content.getError();
break;
}
@ -367,7 +407,7 @@ public class AsyncContentProducerTest
readBytes += b.length;
content.getByteBuffer().get(b);
consumedString += new String(b, StandardCharsets.ISO_8859_1);
content.getByteBuffer().position(content.getByteBuffer().position() + content.remaining());
content.skip(content.remaining());
}
assertThat(nextContentCount, is(totalContentCount));
@ -420,25 +460,19 @@ public class AsyncContentProducerTest
}
}
private Throwable getError(Content.Chunk chunk)
private class ContentListHttpChannel extends HttpChannel
{
return chunk instanceof Content.Chunk.Error error ? error.getCause() : null;
}
private static class ContentListHttpChannel extends HttpChannel
{
private final List<Content.Chunk> contents;
private final Content.Chunk finalContent;
private final List<HttpInput.Content> contents;
private final HttpInput.Content finalContent;
private int index;
public ContentListHttpChannel(List<Content.Chunk> contents, Content.Chunk finalContent)
public ContentListHttpChannel(List<HttpInput.Content> contents, HttpInput.Content finalContent)
{
super(null, null);
super(_contextHandler, new MockConnectionMetaData(_connector));
this.contents = contents;
this.finalContent = finalContent;
}
/*
@Override
public boolean needContent()
{
@ -446,9 +480,9 @@ public class AsyncContentProducerTest
}
@Override
public Content produceContent()
public HttpInput.Content produceContent()
{
Content c;
HttpInput.Content c;
if (index < contents.size())
c = contents.get(index++);
else
@ -456,7 +490,11 @@ public class AsyncContentProducerTest
return c;
}
*/
@Override
public boolean failAllContent(Throwable failure)
{
return false;
}
@Override
public boolean failed(Throwable failure)
@ -471,23 +509,23 @@ public class AsyncContentProducerTest
}
}
private static class ArrayDelayedHttpChannel extends HttpChannel
private class ArrayDelayedHttpChannel extends HttpChannel
{
private final ByteBuffer[] byteBuffers;
private final Content.Chunk finalContent;
private final ScheduledExecutorService scheduledExecutorService;
private final HttpInput.Content finalContent;
private final Scheduler scheduler;
private final CyclicBarrier barrier;
private int counter;
private volatile Content.Chunk nextContent;
private volatile HttpInput.Content nextContent;
public ArrayDelayedHttpChannel(ByteBuffer[] byteBuffers, Content.Chunk finalContent, ScheduledExecutorService scheduledExecutorService, CyclicBarrier barrier)
public ArrayDelayedHttpChannel(ByteBuffer[] byteBuffers, HttpInput.Content finalContent, Scheduler scheduler, CyclicBarrier barrier)
{
super(new ContextHandler(), new MockConnectionMetaData(new MockConnector(new Server())));
super(_contextHandler, new MockConnectionMetaData(_connector));
getContextHandler().setServer(getConnectionMetaData().getConnector().getServer());
this.byteBuffers = new ByteBuffer[byteBuffers.length];
this.finalContent = finalContent;
this.scheduledExecutorService = scheduledExecutorService;
this.scheduler = scheduler;
this.barrier = barrier;
for (int i = 0; i < byteBuffers.length; i++)
{
@ -495,16 +533,15 @@ public class AsyncContentProducerTest
}
}
/*
@Override
public boolean needContent()
{
if (nextContent != null)
return true;
scheduledExecutorService.schedule(() ->
scheduler.schedule(() ->
{
if (byteBuffers.length > counter)
nextContent = new Content.Buffer(byteBuffers[counter++]);
nextContent = new HttpInput.Content(byteBuffers[counter++]);
else
nextContent = finalContent;
try
@ -520,9 +557,9 @@ public class AsyncContentProducerTest
}
@Override
public Content produceContent()
public HttpInput.Content produceContent()
{
Content result = nextContent;
HttpInput.Content result = nextContent;
nextContent = null;
return result;
}
@ -534,7 +571,6 @@ public class AsyncContentProducerTest
counter = byteBuffers.length;
return false;
}
*/
@Override
public boolean failed(Throwable x)
@ -560,7 +596,7 @@ public class AsyncContentProducerTest
}
@Override
public Content.Chunk readFrom(Content.Chunk content)
public HttpInput.Content readFrom(HttpInput.Content content)
{
return null;
}
@ -568,12 +604,12 @@ public class AsyncContentProducerTest
private static class AccountingInterceptor implements HttpInput.Interceptor
{
private final List<Content.Chunk> contents = new ArrayList<>();
private final List<HttpInput.Content> contents = new ArrayList<>();
@Override
public Content.Chunk readFrom(Content.Chunk content)
public HttpInput.Content readFrom(HttpInput.Content content)
{
if (content.isTerminal() || !contents.contains(content))
if (content.isSpecial() || !contents.contains(content))
contents.add(content);
return content;
}

View File

@ -13,25 +13,20 @@
package org.eclipse.jetty.ee9.nested;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.zip.GZIPOutputStream;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EofException;
import org.eclipse.jetty.util.component.Destroyable;
import org.eclipse.jetty.util.compression.InflaterPool;
import org.eclipse.jetty.util.thread.AutoLock;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import static org.hamcrest.MatcherAssert.assertThat;
@ -39,17 +34,14 @@ import static org.hamcrest.Matchers.endsWith;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.nullValue;
@Disabled
public class BlockingContentProducerTest
{
private ScheduledExecutorService scheduledExecutorService;
private InflaterPool inflaterPool;
@BeforeEach
public void setUp()
{
scheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
inflaterPool = new InflaterPool(-1, true);
}
@AfterEach
@ -85,7 +77,7 @@ public class BlockingContentProducerTest
final String originalContentString = asString(buffers);
ContentListener contentListener = new ContentListener();
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, Content.Chunk.EOF, scheduledExecutorService, contentListener);
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, new HttpInput.EofContent(), scheduledExecutorService, contentListener);
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(httpChannel));
contentListener.setContentProducer(contentProducer);
@ -108,7 +100,7 @@ public class BlockingContentProducerTest
final Throwable expectedError = new EofException("Early EOF");
ContentListener contentListener = new ContentListener();
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, Content.Chunk.from(expectedError), scheduledExecutorService, contentListener);
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, new HttpInput.ErrorContent(expectedError), scheduledExecutorService, contentListener);
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(httpChannel));
contentListener.setContentProducer(contentProducer);
@ -130,7 +122,7 @@ public class BlockingContentProducerTest
final String originalContentString = asString(buffers);
ContentListener contentListener = new ContentListener();
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, Content.Chunk.EOF, scheduledExecutorService, contentListener);
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, new HttpInput.EofContent(), scheduledExecutorService, contentListener);
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(httpChannel));
contentListener.setContentProducer(contentProducer);
AccountingInterceptor interceptor = new AccountingInterceptor();
@ -141,17 +133,17 @@ public class BlockingContentProducerTest
Throwable error = readAndAssertContent(totalContentBytesCount, originalContentString, buffers.length + 1, contentProducer);
assertThat(error, nullValue());
Content.Chunk lastContent = contentProducer.nextContent();
assertThat(lastContent.isTerminal(), is(true));
assertThat(lastContent.isLast(), is(true));
HttpInput.Content lastContent = contentProducer.nextContent();
assertThat(lastContent.isSpecial(), is(true));
assertThat(lastContent.isEof(), is(true));
}
assertThat(interceptor.contents.size(), is(4));
assertThat(interceptor.contents.get(0).isTerminal(), is(false));
assertThat(interceptor.contents.get(1).isTerminal(), is(false));
assertThat(interceptor.contents.get(2).isTerminal(), is(false));
assertThat(interceptor.contents.get(3).isTerminal(), is(true));
assertThat(interceptor.contents.get(3).isLast(), is(true));
assertThat(interceptor.contents.get(0).isSpecial(), is(false));
assertThat(interceptor.contents.get(1).isSpecial(), is(false));
assertThat(interceptor.contents.get(2).isSpecial(), is(false));
assertThat(interceptor.contents.get(3).isSpecial(), is(true));
assertThat(interceptor.contents.get(3).isEof(), is(true));
}
@Test
@ -165,7 +157,7 @@ public class BlockingContentProducerTest
final String originalContentString = asString(buffers);
ContentListener contentListener = new ContentListener();
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, Content.Chunk.from(new Throwable("testBlockingContentProducerErrorContentIsPassedToInterceptor error")), scheduledExecutorService, contentListener);
ArrayDelayedHttpChannel httpChannel = new ArrayDelayedHttpChannel(buffers, new HttpInput.ErrorContent(new Throwable("testBlockingContentProducerErrorContentIsPassedToInterceptor error")), scheduledExecutorService, contentListener);
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(httpChannel));
contentListener.setContentProducer(contentProducer);
AccountingInterceptor interceptor = new AccountingInterceptor();
@ -176,37 +168,44 @@ public class BlockingContentProducerTest
Throwable error = readAndAssertContent(totalContentBytesCount, originalContentString, buffers.length + 1, contentProducer);
assertThat(error.getMessage(), is("testBlockingContentProducerErrorContentIsPassedToInterceptor error"));
Content.Chunk lastContent = contentProducer.nextContent();
assertThat(lastContent.isTerminal(), is(true));
assertThat(getError(lastContent).getMessage(), is("testBlockingContentProducerErrorContentIsPassedToInterceptor error"));
HttpInput.Content lastContent = contentProducer.nextContent();
assertThat(lastContent.isSpecial(), is(true));
assertThat(lastContent.getError().getMessage(), is("testBlockingContentProducerErrorContentIsPassedToInterceptor error"));
}
assertThat(interceptor.contents.size(), is(4));
assertThat(interceptor.contents.get(0).isTerminal(), is(false));
assertThat(interceptor.contents.get(1).isTerminal(), is(false));
assertThat(interceptor.contents.get(2).isTerminal(), is(false));
assertThat(interceptor.contents.get(3).isTerminal(), is(true));
assertThat(getError(interceptor.contents.get(3)).getMessage(), is("testBlockingContentProducerErrorContentIsPassedToInterceptor error"));
assertThat(interceptor.contents.get(0).isSpecial(), is(false));
assertThat(interceptor.contents.get(1).isSpecial(), is(false));
assertThat(interceptor.contents.get(2).isSpecial(), is(false));
assertThat(interceptor.contents.get(3).isSpecial(), is(true));
assertThat(interceptor.contents.get(3).getError().getMessage(), is("testBlockingContentProducerErrorContentIsPassedToInterceptor error"));
}
@Test
public void testBlockingContentProducerInterceptorGeneratesError()
{
AtomicInteger contentSucceededCount = new AtomicInteger();
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(Content.Chunk.from(ByteBuffer.allocate(1), false))));
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void succeeded()
{
contentSucceededCount.incrementAndGet();
}
})));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content -> Content.Chunk.from(new Throwable("testBlockingContentProducerInterceptorGeneratesError interceptor error")));
contentProducer.setInterceptor(content -> new HttpInput.ErrorContent(new Throwable("testBlockingContentProducerInterceptorGeneratesError interceptor error")));
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(getError(content1).getMessage(), is("testBlockingContentProducerInterceptorGeneratesError interceptor error"));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.getError().getMessage(), is("testBlockingContentProducerInterceptorGeneratesError interceptor error"));
assertThat(contentProducer.isError(), is(true));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(getError(content2).getMessage(), is("testBlockingContentProducerInterceptorGeneratesError interceptor error"));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.getError().getMessage(), is("testBlockingContentProducerInterceptorGeneratesError interceptor error"));
}
assertThat(contentSucceededCount.get(), is(1));
}
@ -215,20 +214,27 @@ public class BlockingContentProducerTest
public void testBlockingContentProducerInterceptorGeneratesEof()
{
AtomicInteger contentSucceededCount = new AtomicInteger();
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(Content.Chunk.from(ByteBuffer.allocate(1), false))));
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void succeeded()
{
contentSucceededCount.incrementAndGet();
}
})));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content -> Content.Chunk.EOF);
contentProducer.setInterceptor(content -> new HttpInput.EofContent());
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(content1.isLast(), is(true));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.isEof(), is(true));
assertThat(contentProducer.isError(), is(false));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(content2.isLast(), is(true));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.isEof(), is(true));
}
assertThat(contentSucceededCount.get(), is(1));
}
@ -237,7 +243,14 @@ public class BlockingContentProducerTest
public void testBlockingContentProducerInterceptorThrows()
{
AtomicInteger contentFailedCount = new AtomicInteger();
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(Content.Chunk.from(ByteBuffer.allocate(1), false))));
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void failed(Throwable x)
{
contentFailedCount.incrementAndGet();
}
})));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content ->
@ -245,15 +258,15 @@ public class BlockingContentProducerTest
throw new RuntimeException("testBlockingContentProducerInterceptorThrows error");
});
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(getError(content1).getCause().getMessage(), is("testBlockingContentProducerInterceptorThrows error"));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.getError().getCause().getMessage(), is("testBlockingContentProducerInterceptorThrows error"));
assertThat(contentProducer.isError(), is(true));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(getError(content2).getCause().getMessage(), is("testBlockingContentProducerInterceptorThrows error"));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.getError().getCause().getMessage(), is("testBlockingContentProducerInterceptorThrows error"));
}
assertThat(contentFailedCount.get(), is(1));
}
@ -262,18 +275,25 @@ public class BlockingContentProducerTest
public void testBlockingContentProducerInterceptorDoesNotConsume()
{
AtomicInteger contentFailedCount = new AtomicInteger();
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(Content.Chunk.from(ByteBuffer.allocate(1), false))));
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(new HttpInput.Content(ByteBuffer.allocate(1))
{
@Override
public void failed(Throwable x)
{
contentFailedCount.incrementAndGet();
}
})));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content -> null);
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(getError(content1).getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.getError().getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(getError(content2).getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.getError().getMessage(), endsWith("did not consume any of the 1 remaining byte(s) of content"));
}
assertThat(contentFailedCount.get(), is(1));
}
@ -284,12 +304,19 @@ public class BlockingContentProducerTest
AtomicInteger contentSucceededCount = new AtomicInteger();
AtomicInteger specialContentInterceptedCount = new AtomicInteger();
AtomicInteger nullContentInterceptedCount = new AtomicInteger();
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(Content.Chunk.from(ByteBuffer.allocate(0), false))));
ContentProducer contentProducer = new BlockingContentProducer(new AsyncContentProducer(new StaticContentHttpChannel(new HttpInput.Content(ByteBuffer.allocate(0))
{
@Override
public void succeeded()
{
contentSucceededCount.incrementAndGet();
}
})));
try (AutoLock ignored = contentProducer.lock())
{
contentProducer.setInterceptor(content ->
{
if (content.isTerminal())
if (content.isSpecial())
{
specialContentInterceptedCount.incrementAndGet();
return content;
@ -298,13 +325,13 @@ public class BlockingContentProducerTest
return null;
});
Content.Chunk content1 = contentProducer.nextContent();
assertThat(content1.isTerminal(), is(true));
assertThat(content1.isLast(), is(true));
HttpInput.Content content1 = contentProducer.nextContent();
assertThat(content1.isSpecial(), is(true));
assertThat(content1.isEof(), is(true));
Content.Chunk content2 = contentProducer.nextContent();
assertThat(content2.isTerminal(), is(true));
assertThat(content2.isLast(), is(true));
HttpInput.Content content2 = contentProducer.nextContent();
assertThat(content2.isSpecial(), is(true));
assertThat(content2.isEof(), is(true));
}
assertThat(contentSucceededCount.get(), is(1));
assertThat(specialContentInterceptedCount.get(), is(1));
@ -319,14 +346,14 @@ public class BlockingContentProducerTest
Throwable error = null;
while (true)
{
Content.Chunk content = contentProducer.nextContent();
HttpInput.Content content = contentProducer.nextContent();
nextContentCount++;
if (content.isTerminal())
if (content.isSpecial())
{
if (content.isLast())
if (content.isEof())
break;
error = getError(content);
error = content.getError();
break;
}
@ -364,31 +391,6 @@ public class BlockingContentProducerTest
return sb.toString();
}
private static ByteBuffer gzipByteBuffer(ByteBuffer uncompressedBuffer)
{
try
{
ByteArrayOutputStream baos = new ByteArrayOutputStream();
GZIPOutputStream output = new GZIPOutputStream(baos);
byte[] b = new byte[uncompressedBuffer.remaining()];
uncompressedBuffer.get(b);
output.write(b);
output.close();
return ByteBuffer.wrap(baos.toByteArray());
}
catch (IOException e)
{
throw new RuntimeException(e);
}
}
private Throwable getError(Content.Chunk chunk)
{
return chunk instanceof Content.Chunk.Error error ? error.getCause() : null;
}
private static class ContentListener
{
private ContentProducer contentProducer;
@ -413,15 +415,14 @@ public class BlockingContentProducerTest
private static class StaticContentHttpChannel extends HttpChannel
{
private Content.Chunk content;
private HttpInput.Content content;
public StaticContentHttpChannel(Content.Chunk content)
public StaticContentHttpChannel(HttpInput.Content content)
{
super(null, null);
super(new ContextHandler(), new MockConnectionMetaData(new MockConnector()));
this.content = content;
}
/*
@Override
public boolean needContent()
{
@ -429,10 +430,10 @@ public class BlockingContentProducerTest
}
@Override
public Content produceContent()
public HttpInput.Content produceContent()
{
Content c = content;
content = Content.EOF;
HttpInput.Content c = content;
content = new HttpInput.EofContent();
return c;
}
@ -442,8 +443,6 @@ public class BlockingContentProducerTest
return false;
}
*/
@Override
public boolean failed(Throwable failure)
{
@ -460,15 +459,15 @@ public class BlockingContentProducerTest
private static class ArrayDelayedHttpChannel extends HttpChannel
{
private final ByteBuffer[] byteBuffers;
private final Content.Chunk finalContent;
private final HttpInput.Content finalContent;
private final ScheduledExecutorService scheduledExecutorService;
private final ContentListener contentListener;
private int counter;
private volatile Content.Chunk nextContent;
private volatile HttpInput.Content nextContent;
public ArrayDelayedHttpChannel(ByteBuffer[] byteBuffers, Content.Chunk finalContent, ScheduledExecutorService scheduledExecutorService, ContentListener contentListener)
public ArrayDelayedHttpChannel(ByteBuffer[] byteBuffers, HttpInput.Content finalContent, ScheduledExecutorService scheduledExecutorService, ContentListener contentListener)
{
super(null, null);
super(new ContextHandler(), new MockConnectionMetaData(new MockConnector()));
this.byteBuffers = new ByteBuffer[byteBuffers.length];
this.finalContent = finalContent;
this.scheduledExecutorService = scheduledExecutorService;
@ -479,7 +478,6 @@ public class BlockingContentProducerTest
}
}
/*
@Override
public boolean needContent()
{
@ -488,7 +486,7 @@ public class BlockingContentProducerTest
scheduledExecutorService.schedule(() ->
{
if (byteBuffers.length > counter)
nextContent = new Content.Buffer(byteBuffers[counter++]);
nextContent = new HttpInput.Content(byteBuffers[counter++]);
else
nextContent = finalContent;
contentListener.onContent();
@ -497,14 +495,13 @@ public class BlockingContentProducerTest
}
@Override
public Content produceContent()
public HttpInput.Content produceContent()
{
Content result = nextContent;
HttpInput.Content result = nextContent;
nextContent = null;
return result;
}
@Override
public boolean failAllContent(Throwable failure)
{
@ -512,7 +509,6 @@ public class BlockingContentProducerTest
counter = byteBuffers.length;
return false;
}
*/
@Override
public boolean failed(Throwable x)
@ -538,7 +534,7 @@ public class BlockingContentProducerTest
}
@Override
public Content.Chunk readFrom(Content.Chunk content)
public HttpInput.Content readFrom(HttpInput.Content content)
{
return null;
}
@ -546,10 +542,10 @@ public class BlockingContentProducerTest
private static class AccountingInterceptor implements HttpInput.Interceptor
{
private List<Content.Chunk> contents = new ArrayList<>();
private final List<HttpInput.Content> contents = new ArrayList<>();
@Override
public Content.Chunk readFrom(Content.Chunk content)
public HttpInput.Content readFrom(HttpInput.Content content)
{
if (!contents.contains(content))
contents.add(content);

View File

@ -172,9 +172,8 @@ public class HttpServerTestFixture
int r = c.remaining();
c.get(content, offset, r);
offset += r;
c.release();
}
c.release();
if (c.isLast())
break;
}

View File

@ -324,12 +324,11 @@ public class RequestTest
_handler._checker = (request, response) ->
{
request.getParameterMap();
// should have thrown a BadMessageException
//
return false;
};
//Send a request with query string with illegal hex code to cause
//an exception parsing the params
//Send a request with a form body that is smaller than Content-Length.
String request = "POST / HTTP/1.1\r\n" +
"Host: whatever\r\n" +
"Content-Type: " + MimeTypes.Type.FORM_ENCODED.asString() + "\n" +
@ -1820,7 +1819,7 @@ public class RequestTest
@Test
public void testNotSupportedCharacterEncoding()
{
Request request = new Request(new HttpChannel(_context, new MockConnectionMetaData(new MockConnector())), null);
Request request = new Request(new HttpChannel(_context, new MockConnectionMetaData(_connector)), null);
assertThrows(UnsupportedEncodingException.class, () -> request.setCharacterEncoding("doesNotExist"));
}
@ -1828,7 +1827,7 @@ public class RequestTest
public void testGetterSafeFromNullPointerException()
{
// This is only needed for tests that mock with null values.
Request request = new Request(new HttpChannel(_context, new MockConnectionMetaData(new MockConnector())), null);
Request request = new Request(new HttpChannel(_context, new MockConnectionMetaData(_connector)), null);
assertNull(request.getAuthType());
assertNull(request.getAuthentication());
@ -1869,7 +1868,7 @@ public class RequestTest
public void testPushBuilder()
{
String uri = "http://host/foo/something";
HttpChannel httpChannel = new HttpChannel(_context, new MockConnectionMetaData(new MockConnector()));
HttpChannel httpChannel = new HttpChannel(_context, new MockConnectionMetaData(_connector));
Request request = new MockRequest(httpChannel, new HttpInput(httpChannel));
request.getResponse().getHttpFields().add(new HttpCookie.SetCookieHttpField(new HttpCookie("good", "thumbsup", 100), CookieCompliance.RFC6265));
request.getResponse().getHttpFields().add(new HttpCookie.SetCookieHttpField(new HttpCookie("bonza", "bewdy", 1), CookieCompliance.RFC6265));
@ -1909,7 +1908,7 @@ public class RequestTest
public void testPushBuilderWithIdNoAuth()
{
String uri = "http://host/foo/something";
HttpChannel httpChannel = new HttpChannel(_context, new MockConnectionMetaData(new MockConnector()));
HttpChannel httpChannel = new HttpChannel(_context, new MockConnectionMetaData(_connector));
Request request = new MockRequest(httpChannel, new HttpInput(httpChannel))
{
@Override

View File

@ -439,6 +439,8 @@ public class AsyncMiddleManServlet extends AbstractProxyServlet
@Override
public void onContent(Response serverResponse, Content.Chunk chunk, Runnable demander)
{
if (chunk.canRetain())
chunk.retain();
Callback callback = Callback.from(chunk::release, Callback.from(demander, serverResponse::abort));
try
{

View File

@ -216,7 +216,8 @@ public class ProxyServlet extends AbstractProxyServlet
content.get(buffer);
offset = 0;
}
if (chunk.canRetain())
chunk.retain();
Callback callback = Callback.from(chunk::release, Callback.from(demander, proxyResponse::abort));
onResponseContent(request, response, proxyResponse, buffer, offset, length, callback);
}

View File

@ -63,7 +63,6 @@ import org.eclipse.jetty.http2.api.Session;
import org.eclipse.jetty.http2.client.transport.internal.HttpConnectionOverHTTP2;
import org.eclipse.jetty.http2.internal.HTTP2Session;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EofException;
import org.eclipse.jetty.logging.StacklessLogging;
import org.eclipse.jetty.server.internal.HttpChannelState;
@ -1211,71 +1210,71 @@ public class AsyncIOServletTest extends AbstractTest
httpInput.addInterceptor(new HttpInput.Interceptor()
{
int state = 0;
Content.Chunk saved;
HttpInput.Content saved;
@Override
public Content.Chunk readFrom(Content.Chunk chunk)
public HttpInput.Content readFrom(HttpInput.Content content)
{
switch (state)
{
case 0:
// null transform
chunk.skip(chunk.remaining());
chunk.release();
content.skip(content.remaining());
content.succeeded();
state++;
return null;
case 1:
{
// copy transform
if (!chunk.hasRemaining())
if (!content.hasContent())
{
state++;
return chunk;
return content;
}
ByteBuffer copy = wrap(toArray(chunk.getByteBuffer()));
chunk.skip(copy.remaining());
chunk.release();
return Content.Chunk.from(copy, false);
ByteBuffer copy = wrap(toArray(content.getByteBuffer()));
content.skip(copy.remaining());
content.succeeded();
return new HttpInput.Content(copy);
}
case 2:
// byte by byte
if (!chunk.hasRemaining())
if (!content.hasContent())
{
state++;
return chunk;
return content;
}
byte[] b = new byte[1];
int l = chunk.get(b, 0, 1);
if (!chunk.hasRemaining())
chunk.release();
return Content.Chunk.from(wrap(b, 0, l), false);
int l = content.get(b, 0, 1);
if (!content.hasContent())
content.succeeded();
return new HttpInput.Content(wrap(b, 0, l));
case 3:
{
// double vision
if (!chunk.hasRemaining())
if (!content.hasContent())
{
if (saved == null)
{
state++;
return chunk;
return content;
}
Content.Chunk ref = saved;
HttpInput.Content ref = saved;
saved = null;
return ref;
}
byte[] data = toArray(chunk.getByteBuffer());
chunk.skip(data.length);
chunk.release();
saved = Content.Chunk.from(wrap(data), false);
return Content.Chunk.from(wrap(data), false);
byte[] data = toArray(content.getByteBuffer());
content.skip(data.length);
content.succeeded();
saved = new HttpInput.Content(wrap(data));
return new HttpInput.Content(wrap(data));
}
default:
return chunk;
return content;
}
}
});
@ -1560,7 +1559,7 @@ public class AsyncIOServletTest extends AbstractTest
HttpInput httpInput = ((org.eclipse.jetty.ee9.nested.Request)request).getHttpInput();
httpInput.addInterceptor(chunk ->
{
if (!chunk.hasRemaining())
if (!chunk.hasContent())
return chunk;
// skip contents with odd numbers
@ -1568,16 +1567,16 @@ public class AsyncIOServletTest extends AbstractTest
duplicate.get();
byte integer = duplicate.get();
int idx = Character.getNumericValue(integer);
Content.Chunk chunkCopy = Content.Chunk.from(chunk.getByteBuffer().duplicate(), false);
HttpInput.Content chunkCopy = new HttpInput.Content(chunk.getByteBuffer().duplicate());
chunk.skip(chunk.remaining());
chunk.release();
chunk.succeeded();
if (idx % 2 == 0)
return chunkCopy;
return null;
});
httpInput.addInterceptor(chunk ->
{
if (!chunk.hasRemaining())
if (!chunk.hasContent())
return chunk;
// reverse the bytes
@ -1585,7 +1584,7 @@ public class AsyncIOServletTest extends AbstractTest
byte[] bytes = new byte[2];
bytes[1] = byteBuffer.get();
bytes[0] = byteBuffer.get();
return Content.Chunk.from(wrap(bytes), false);
return new HttpInput.Content(wrap(bytes));
});
AsyncContext asyncContext = request.startAsync();

View File

@ -600,7 +600,6 @@ public class ServerTimeoutsTest extends AbstractTest
.onResponseContentAsync((response, chunk, demander) ->
{
objects.offer(chunk.remaining());
chunk.release();
objects.offer(demander);
})
.send(result ->