listeners)
{
@@ -97,6 +99,19 @@ public class HttpResponse implements Response
return result;
}
+ public HttpFields getTrailers()
+ {
+ return trailers;
+ }
+
+ public HttpResponse trailer(HttpField trailer)
+ {
+ if (trailers == null)
+ trailers = new HttpFields();
+ trailers.add(trailer);
+ return this;
+ }
+
@Override
public boolean abort(Throwable cause)
{
diff --git a/jetty-client/src/main/java/org/eclipse/jetty/client/HttpSender.java b/jetty-client/src/main/java/org/eclipse/jetty/client/HttpSender.java
index 7d67bc5940e..bdae1570819 100644
--- a/jetty-client/src/main/java/org/eclipse/jetty/client/HttpSender.java
+++ b/jetty-client/src/main/java/org/eclipse/jetty/client/HttpSender.java
@@ -20,10 +20,12 @@ package org.eclipse.jetty.client;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
import org.eclipse.jetty.client.api.ContentProvider;
import org.eclipse.jetty.client.api.Request;
import org.eclipse.jetty.client.api.Result;
+import org.eclipse.jetty.http.HttpFields;
import org.eclipse.jetty.http.HttpHeader;
import org.eclipse.jetty.http.HttpHeaderValue;
import org.eclipse.jetty.util.BufferUtil;
@@ -31,7 +33,6 @@ import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.IteratingCallback;
import org.eclipse.jetty.util.log.Log;
import org.eclipse.jetty.util.log.Logger;
-import org.eclipse.jetty.util.thread.Invocable.InvocationType;
/**
* {@link HttpSender} abstracts the algorithm to send HTTP requests, so that subclasses only implement
@@ -42,9 +43,9 @@ import org.eclipse.jetty.util.thread.Invocable.InvocationType;
* {@link HttpSender} governs two state machines.
*
* The request state machine is updated by {@link HttpSender} as the various steps of sending a request
- * are executed, see RequestState
.
+ * are executed, see {@code RequestState}.
* At any point in time, a user thread may abort the request, which may (if the request has not been
- * completely sent yet) move the request state machine to RequestState#FAILURE
.
+ * completely sent yet) move the request state machine to {@code RequestState#FAILURE}.
* The request state machine guarantees that the request steps are executed (by I/O threads) only if
* the request has not been failed already.
*
@@ -64,7 +65,8 @@ public abstract class HttpSender implements AsyncContentProvider.Listener
private final AtomicReference senderState = new AtomicReference<>(SenderState.IDLE);
private final Callback commitCallback = new CommitCallback();
private final IteratingCallback contentCallback = new ContentCallback();
- private final Callback lastCallback = new LastContentCallback();
+ private final Callback trailersCallback = new TrailersCallback();
+ private final Callback lastCallback = new LastCallback();
private final HttpChannel channel;
private HttpContent content;
private Throwable failure;
@@ -407,7 +409,7 @@ public abstract class HttpSender implements AsyncContentProvider.Listener
/**
* Implementations should send the content at the {@link HttpContent} cursor position over the wire.
*
- * The {@link HttpContent} cursor is advanced by {@link HttpSender} at the right time, and if more
+ * The {@link HttpContent} cursor is advanced by HttpSender at the right time, and if more
* content needs to be sent, this method is invoked again; subclasses need only to send the content
* at the {@link HttpContent} cursor position.
*
@@ -422,6 +424,15 @@ public abstract class HttpSender implements AsyncContentProvider.Listener
*/
protected abstract void sendContent(HttpExchange exchange, HttpContent content, Callback callback);
+ /**
+ * Implementations should send the HTTP trailers and notify the given {@code callback} of the
+ * result of this operation.
+ *
+ * @param exchange the exchange to send
+ * @param callback the callback to notify
+ */
+ protected abstract void sendTrailers(HttpExchange exchange, Callback callback);
+
protected void reset()
{
HttpContent content = this.content;
@@ -674,13 +685,6 @@ public abstract class HttpSender implements AsyncContentProvider.Listener
private class CommitCallback implements Callback
{
-
- @Override
- public InvocationType getInvocationType()
- {
- return content.getInvocationType();
- }
-
@Override
public void succeeded()
{
@@ -721,10 +725,20 @@ public abstract class HttpSender implements AsyncContentProvider.Listener
if (content == null)
return;
- if (!content.hasContent())
+ HttpRequest request = exchange.getRequest();
+ Supplier trailers = request.getTrailers();
+ boolean hasContent = content.hasContent();
+ if (!hasContent)
{
- // No content to send, we are done.
- someToSuccess(exchange);
+ if (trailers == null)
+ {
+ // No trailers or content to send, we are done.
+ someToSuccess(exchange);
+ }
+ else
+ {
+ sendTrailers(exchange, lastCallback);
+ }
}
else
{
@@ -825,7 +839,9 @@ public abstract class HttpSender implements AsyncContentProvider.Listener
if (lastContent)
{
- sendContent(exchange, content, lastCallback);
+ HttpRequest request = exchange.getRequest();
+ Supplier trailers = request.getTrailers();
+ sendContent(exchange, content, trailers == null ? lastCallback : trailersCallback);
return Action.IDLE;
}
@@ -884,19 +900,35 @@ public abstract class HttpSender implements AsyncContentProvider.Listener
@Override
protected void onCompleteSuccess()
{
- // Nothing to do, since we always return false from process().
- // Termination is obtained via LastContentCallback.
+ // Nothing to do, since we always return IDLE from process().
+ // Termination is obtained via LastCallback.
}
}
- private class LastContentCallback implements Callback
+ private class TrailersCallback implements Callback
{
@Override
- public InvocationType getInvocationType()
+ public void succeeded()
{
- return content.getInvocationType();
+ HttpExchange exchange = getHttpExchange();
+ if (exchange == null)
+ return;
+ sendTrailers(exchange, lastCallback);
}
+ @Override
+ public void failed(Throwable x)
+ {
+ HttpContent content = HttpSender.this.content;
+ if (content == null)
+ return;
+ content.failed(x);
+ anyToFailure(x);
+ }
+ }
+
+ private class LastCallback implements Callback
+ {
@Override
public void succeeded()
{
diff --git a/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpReceiverOverHTTP.java b/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpReceiverOverHTTP.java
index 3239f703b89..3c19684e930 100644
--- a/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpReceiverOverHTTP.java
+++ b/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpReceiverOverHTTP.java
@@ -276,7 +276,17 @@ public class HttpReceiverOverHTTP extends HttpReceiver implements HttpParser.Res
{
return false;
}
-
+
+ @Override
+ public void parsedTrailer(HttpField trailer)
+ {
+ HttpExchange exchange = getHttpExchange();
+ if (exchange == null)
+ return;
+
+ exchange.getResponse().trailer(trailer);
+ }
+
@Override
public boolean messageComplete()
{
diff --git a/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpSenderOverHTTP.java b/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpSenderOverHTTP.java
index d3976261831..a6b0b40cef0 100644
--- a/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpSenderOverHTTP.java
+++ b/jetty-client/src/main/java/org/eclipse/jetty/client/http/HttpSenderOverHTTP.java
@@ -23,26 +23,29 @@ import java.nio.ByteBuffer;
import org.eclipse.jetty.client.HttpClient;
import org.eclipse.jetty.client.HttpContent;
import org.eclipse.jetty.client.HttpExchange;
+import org.eclipse.jetty.client.HttpRequest;
import org.eclipse.jetty.client.HttpRequestException;
import org.eclipse.jetty.client.HttpSender;
import org.eclipse.jetty.client.api.ContentProvider;
-import org.eclipse.jetty.client.api.Request;
import org.eclipse.jetty.http.HttpGenerator;
import org.eclipse.jetty.http.HttpURI;
import org.eclipse.jetty.http.MetaData;
import org.eclipse.jetty.io.ByteBufferPool;
import org.eclipse.jetty.io.EndPoint;
+import org.eclipse.jetty.util.BufferUtil;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.IteratingCallback;
public class HttpSenderOverHTTP extends HttpSender
{
private final HttpGenerator generator = new HttpGenerator();
+ private final HttpClient httpClient;
private boolean shutdown;
public HttpSenderOverHTTP(HttpChannelOverHTTP channel)
{
super(channel);
+ httpClient = channel.getHttpDestination().getHttpClient();
}
@Override
@@ -71,8 +74,7 @@ public class HttpSenderOverHTTP extends HttpSender
{
try
{
- HttpClient client = getHttpChannel().getHttpDestination().getHttpClient();
- ByteBufferPool bufferPool = client.getByteBufferPool();
+ ByteBufferPool bufferPool = httpClient.getByteBufferPool();
ByteBuffer chunk = null;
while (true)
{
@@ -90,6 +92,11 @@ public class HttpSenderOverHTTP extends HttpSender
chunk = bufferPool.acquire(HttpGenerator.CHUNK_SIZE, false);
break;
}
+ case NEED_CHUNK_TRAILER:
+ {
+ callback.succeeded();
+ return;
+ }
case FLUSH:
{
EndPoint endPoint = getHttpChannel().getHttpConnection().getEndPoint();
@@ -131,6 +138,21 @@ public class HttpSenderOverHTTP extends HttpSender
}
}
+ @Override
+ protected void sendTrailers(HttpExchange exchange, Callback callback)
+ {
+ try
+ {
+ new TrailersCallback(callback).iterate();
+ }
+ catch (Throwable x)
+ {
+ if (LOG.isDebugEnabled())
+ LOG.debug(x);
+ callback.failed(x);
+ }
+ }
+
@Override
protected void reset()
{
@@ -181,7 +203,7 @@ public class HttpSenderOverHTTP extends HttpSender
this.exchange = exchange;
this.callback = callback;
- Request request = exchange.getRequest();
+ HttpRequest request = exchange.getRequest();
ContentProvider requestContent = request.getContent();
long contentLength = requestContent == null ? -1 : requestContent.getLength();
String path = request.getPath();
@@ -189,6 +211,7 @@ public class HttpSenderOverHTTP extends HttpSender
if (query != null)
path += "?" + query;
metaData = new MetaData.Request(request.getMethod(), new HttpURI(path), request.getVersion(), request.getHeaders(), contentLength);
+ metaData.setTrailerSupplier(request.getTrailers());
if (!expects100Continue(request))
{
@@ -201,9 +224,6 @@ public class HttpSenderOverHTTP extends HttpSender
@Override
protected Action process() throws Exception
{
- HttpClient client = getHttpChannel().getHttpDestination().getHttpClient();
- ByteBufferPool bufferPool = client.getByteBufferPool();
-
while (true)
{
HttpGenerator.Result result = generator.generateRequest(metaData, headerBuffer, chunkBuffer, contentBuffer, lastContent);
@@ -217,31 +237,28 @@ public class HttpSenderOverHTTP extends HttpSender
{
case NEED_HEADER:
{
- headerBuffer = bufferPool.acquire(client.getRequestBufferSize(), false);
+ headerBuffer = httpClient.getByteBufferPool().acquire(httpClient.getRequestBufferSize(), false);
break;
}
case NEED_CHUNK:
{
- chunkBuffer = bufferPool.acquire(HttpGenerator.CHUNK_SIZE, false);
+ chunkBuffer = httpClient.getByteBufferPool().acquire(HttpGenerator.CHUNK_SIZE, false);
break;
}
+ case NEED_CHUNK_TRAILER:
+ {
+ return Action.SUCCEEDED;
+ }
case FLUSH:
{
EndPoint endPoint = getHttpChannel().getHttpConnection().getEndPoint();
+ if (headerBuffer == null)
+ headerBuffer = BufferUtil.EMPTY_BUFFER;
if (chunkBuffer == null)
- {
- if (contentBuffer == null)
- endPoint.write(this, headerBuffer);
- else
- endPoint.write(this, headerBuffer, contentBuffer);
- }
- else
- {
- if (contentBuffer == null)
- endPoint.write(this, headerBuffer, chunkBuffer);
- else
- endPoint.write(this, headerBuffer, chunkBuffer, contentBuffer);
- }
+ chunkBuffer = BufferUtil.EMPTY_BUFFER;
+ if (contentBuffer == null)
+ contentBuffer = BufferUtil.EMPTY_BUFFER;
+ endPoint.write(this, headerBuffer, chunkBuffer, contentBuffer);
generated = true;
return Action.SCHEDULED;
}
@@ -296,13 +313,91 @@ public class HttpSenderOverHTTP extends HttpSender
private void release()
{
- HttpClient client = getHttpChannel().getHttpDestination().getHttpClient();
- ByteBufferPool bufferPool = client.getByteBufferPool();
- bufferPool.release(headerBuffer);
+ ByteBufferPool bufferPool = httpClient.getByteBufferPool();
+ if (headerBuffer != BufferUtil.EMPTY_BUFFER)
+ bufferPool.release(headerBuffer);
headerBuffer = null;
- if (chunkBuffer != null)
+ if (chunkBuffer != BufferUtil.EMPTY_BUFFER)
bufferPool.release(chunkBuffer);
chunkBuffer = null;
+ contentBuffer = null;
+ }
+ }
+
+ private class TrailersCallback extends IteratingCallback
+ {
+ private final Callback callback;
+ private ByteBuffer chunkBuffer;
+
+ public TrailersCallback(Callback callback)
+ {
+ this.callback = callback;
+ }
+
+ @Override
+ protected Action process() throws Throwable
+ {
+ while (true)
+ {
+ HttpGenerator.Result result = generator.generateRequest(null, null, chunkBuffer, null, true);
+ if (LOG.isDebugEnabled())
+ LOG.debug("Generated trailers {}/{}", result, generator);
+ switch (result)
+ {
+ case NEED_CHUNK_TRAILER:
+ {
+ chunkBuffer = httpClient.getByteBufferPool().acquire(httpClient.getRequestBufferSize(), false);
+ break;
+ }
+ case FLUSH:
+ {
+ EndPoint endPoint = getHttpChannel().getHttpConnection().getEndPoint();
+ endPoint.write(this, chunkBuffer);
+ return Action.SCHEDULED;
+ }
+ case SHUTDOWN_OUT:
+ {
+ shutdownOutput();
+ return Action.SUCCEEDED;
+ }
+ case DONE:
+ {
+ return Action.SUCCEEDED;
+ }
+ default:
+ {
+ throw new IllegalStateException(result.toString());
+ }
+ }
+ }
+ }
+
+ @Override
+ public void succeeded()
+ {
+ release();
+ super.succeeded();
+ }
+
+ @Override
+ public void failed(Throwable x)
+ {
+ release();
+ callback.failed(x);
+ super.failed(x);
+ }
+
+ @Override
+ protected void onCompleteSuccess()
+ {
+ super.onCompleteSuccess();
+ callback.succeeded();
+ }
+
+ private void release()
+ {
+ httpClient.getByteBufferPool().release(chunkBuffer);
+ chunkBuffer = null;
}
}
diff --git a/jetty-fcgi/fcgi-client/src/main/java/org/eclipse/jetty/fcgi/client/http/HttpSenderOverFCGI.java b/jetty-fcgi/fcgi-client/src/main/java/org/eclipse/jetty/fcgi/client/http/HttpSenderOverFCGI.java
index 6598cc7a547..2877ae0ee61 100644
--- a/jetty-fcgi/fcgi-client/src/main/java/org/eclipse/jetty/fcgi/client/http/HttpSenderOverFCGI.java
+++ b/jetty-fcgi/fcgi-client/src/main/java/org/eclipse/jetty/fcgi/client/http/HttpSenderOverFCGI.java
@@ -125,4 +125,10 @@ public class HttpSenderOverFCGI extends HttpSender
getHttpChannel().flush(result);
}
}
+
+ @Override
+ protected void sendTrailers(HttpExchange exchange, Callback callback)
+ {
+ callback.succeeded();
+ }
}
diff --git a/jetty-http/src/main/java/org/eclipse/jetty/http/HttpGenerator.java b/jetty-http/src/main/java/org/eclipse/jetty/http/HttpGenerator.java
index 63994a335d1..e8ea548547a 100644
--- a/jetty-http/src/main/java/org/eclipse/jetty/http/HttpGenerator.java
+++ b/jetty-http/src/main/java/org/eclipse/jetty/http/HttpGenerator.java
@@ -18,8 +18,6 @@
package org.eclipse.jetty.http;
-import static org.eclipse.jetty.http.HttpStatus.INTERNAL_SERVER_ERROR_500;
-
import java.io.IOException;
import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
@@ -34,6 +32,8 @@ import org.eclipse.jetty.util.Trie;
import org.eclipse.jetty.util.log.Log;
import org.eclipse.jetty.util.log.Logger;
+import static org.eclipse.jetty.http.HttpStatus.INTERNAL_SERVER_ERROR_500;
+
/**
* HttpGenerator. Builds HTTP Messages.
*
@@ -764,7 +764,7 @@ public class HttpGenerator
}
// Else if we are HTTP/1.1 and the content length is unknown and we are either persistent
// or it is a request with content (which cannot EOF) or the app has requested chunking
- else if (http11 && content_length<0 && (_persistent || assumed_content_request || chunked_hint))
+ else if (http11 && (chunked_hint || content_length<0 && (_persistent || assumed_content_request)))
{
// we use chunking
_endOfContent = EndOfContent.CHUNKED_CONTENT;
diff --git a/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpReceiverOverHTTP2.java b/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpReceiverOverHTTP2.java
index 1bae533c97d..70913728500 100644
--- a/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpReceiverOverHTTP2.java
+++ b/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpReceiverOverHTTP2.java
@@ -71,27 +71,37 @@ public class HttpReceiverOverHTTP2 extends HttpReceiver implements Stream.Listen
if (exchange == null)
return;
- HttpResponse response = exchange.getResponse();
- MetaData.Response metaData = (MetaData.Response)frame.getMetaData();
- response.version(metaData.getHttpVersion()).status(metaData.getStatus()).reason(metaData.getReason());
-
- if (responseBegin(exchange))
+ HttpResponse httpResponse = exchange.getResponse();
+ MetaData metaData = frame.getMetaData();
+ if (metaData.isResponse())
{
- HttpFields headers = metaData.getFields();
- for (HttpField header : headers)
- {
- if (!responseHeader(exchange, header))
- return;
- }
+ MetaData.Response response = (MetaData.Response)frame.getMetaData();
+ httpResponse.version(response.getHttpVersion()).status(response.getStatus()).reason(response.getReason());
- if (responseHeaders(exchange))
+ if (responseBegin(exchange))
{
- int status = metaData.getStatus();
- boolean informational = HttpStatus.isInformational(status) && status != HttpStatus.SWITCHING_PROTOCOLS_101;
- if (frame.isEndStream() || informational)
- responseSuccess(exchange);
+ HttpFields headers = response.getFields();
+ for (HttpField header : headers)
+ {
+ if (!responseHeader(exchange, header))
+ return;
+ }
+
+ if (responseHeaders(exchange))
+ {
+ int status = response.getStatus();
+ boolean informational = HttpStatus.isInformational(status) && status != HttpStatus.SWITCHING_PROTOCOLS_101;
+ if (frame.isEndStream() || informational)
+ responseSuccess(exchange);
+ }
}
}
+ else
+ {
+ HttpFields trailers = metaData.getFields();
+ trailers.forEach(httpResponse::trailer);
+ responseSuccess(exchange);
+ }
}
@Override
diff --git a/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpSenderOverHTTP2.java b/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpSenderOverHTTP2.java
index 6092fe324cd..c2230e75cb9 100644
--- a/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpSenderOverHTTP2.java
+++ b/jetty-http2/http2-http-client-transport/src/main/java/org/eclipse/jetty/http2/client/http/HttpSenderOverHTTP2.java
@@ -19,11 +19,13 @@
package org.eclipse.jetty.http2.client.http;
import java.net.URI;
+import java.util.function.Supplier;
import org.eclipse.jetty.client.HttpContent;
import org.eclipse.jetty.client.HttpExchange;
+import org.eclipse.jetty.client.HttpRequest;
import org.eclipse.jetty.client.HttpSender;
-import org.eclipse.jetty.client.api.Request;
+import org.eclipse.jetty.http.HttpFields;
import org.eclipse.jetty.http.HttpURI;
import org.eclipse.jetty.http.HttpVersion;
import org.eclipse.jetty.http.MetaData;
@@ -49,11 +51,13 @@ public class HttpSenderOverHTTP2 extends HttpSender
@Override
protected void sendHeaders(HttpExchange exchange, final HttpContent content, final Callback callback)
{
- Request request = exchange.getRequest();
+ HttpRequest request = exchange.getRequest();
String path = relativize(request.getPath());
HttpURI uri = new HttpURI(request.getScheme(), request.getHost(), request.getPort(), path, null, request.getQuery(), null);
MetaData.Request metaData = new MetaData.Request(request.getMethod(), uri, HttpVersion.HTTP_2, request.getHeaders());
- HeadersFrame headersFrame = new HeadersFrame(metaData, null, !content.hasContent());
+ Supplier trailers = request.getTrailers();
+ metaData.setTrailerSupplier(trailers);
+ HeadersFrame headersFrame = new HeadersFrame(metaData, null, trailers == null && !content.hasContent());
HttpChannelOverHTTP2 channel = getHttpChannel();
Promise promise = new Promise()
{
@@ -66,7 +70,7 @@ public class HttpSenderOverHTTP2 extends HttpSender
if (content.hasContent() && !expects100Continue(request))
{
boolean advanced = content.advance();
- boolean lastContent = content.isLast();
+ boolean lastContent = trailers == null && content.isLast();
if (advanced || lastContent)
{
DataFrame dataFrame = new DataFrame(stream.getId(), content.getByteBuffer(), lastContent);
@@ -115,8 +119,19 @@ public class HttpSenderOverHTTP2 extends HttpSender
else
{
Stream stream = getHttpChannel().getStream();
- DataFrame frame = new DataFrame(stream.getId(), content.getByteBuffer(), content.isLast());
+ Supplier trailers = exchange.getRequest().getTrailers();
+ DataFrame frame = new DataFrame(stream.getId(), content.getByteBuffer(), trailers == null && content.isLast());
stream.data(frame, callback);
}
}
+
+ @Override
+ protected void sendTrailers(HttpExchange exchange, Callback callback)
+ {
+ Supplier trailers = exchange.getRequest().getTrailers();
+ MetaData metaData = new MetaData(HttpVersion.HTTP_2, trailers.get());
+ Stream stream = getHttpChannel().getStream();
+ HeadersFrame trailersFrame = new HeadersFrame(stream.getId(), metaData, null, true);
+ stream.headers(trailersFrame, callback);
+ }
}
diff --git a/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HTTP2ServerConnection.java b/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HTTP2ServerConnection.java
index c8036f48965..f1b1636b742 100644
--- a/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HTTP2ServerConnection.java
+++ b/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HTTP2ServerConnection.java
@@ -173,7 +173,11 @@ public class HTTP2ServerConnection extends HTTP2Connection implements Connection
LOG.debug("Processing trailers {} on {}", frame, stream);
HttpChannelOverHTTP2 channel = (HttpChannelOverHTTP2)stream.getAttribute(IStream.CHANNEL_ATTRIBUTE);
if (channel != null)
- channel.onRequestTrailers(frame);
+ {
+ Runnable task = channel.onRequestTrailers(frame);
+ if (task != null)
+ offerTask(task, false);
+ }
}
public boolean onStreamTimeout(IStream stream, Throwable failure)
diff --git a/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpChannelOverHTTP2.java b/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpChannelOverHTTP2.java
index 21f463100e1..1a391090806 100644
--- a/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpChannelOverHTTP2.java
+++ b/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpChannelOverHTTP2.java
@@ -282,11 +282,12 @@ public class HttpChannelOverHTTP2 extends HttpChannel
return handle || wasDelayed ? this : null;
}
- public void onRequestTrailers(HeadersFrame frame)
+ public Runnable onRequestTrailers(HeadersFrame frame)
{
HttpFields trailers = frame.getMetaData().getFields();
- onTrailers(trailers);
- onRequestComplete();
+ if (trailers.size() > 0)
+ onTrailers(trailers);
+
if (LOG.isDebugEnabled())
{
Stream stream = getStream();
@@ -294,6 +295,14 @@ public class HttpChannelOverHTTP2 extends HttpChannel
stream.getId(), Integer.toHexString(stream.getSession().hashCode()),
System.lineSeparator(), trailers);
}
+
+ boolean handle = onRequestComplete();
+
+ boolean wasDelayed = _delayedUntilContent;
+ _delayedUntilContent = false;
+ if (wasDelayed)
+ _handled = true;
+ return handle || wasDelayed ? this : null;
}
public boolean isRequestHandled()
diff --git a/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpTransportOverHTTP2.java b/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpTransportOverHTTP2.java
index 680cc93fc91..51eb2a90e26 100644
--- a/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpTransportOverHTTP2.java
+++ b/jetty-http2/http2-server/src/main/java/org/eclipse/jetty/http2/server/HttpTransportOverHTTP2.java
@@ -20,7 +20,9 @@ package org.eclipse.jetty.http2.server;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Supplier;
+import org.eclipse.jetty.http.HttpFields;
import org.eclipse.jetty.http.HttpStatus;
import org.eclipse.jetty.http.HttpVersion;
import org.eclipse.jetty.http.MetaData;
@@ -48,6 +50,7 @@ public class HttpTransportOverHTTP2 implements HttpTransport
private final Connector connector;
private final HTTP2ServerConnection connection;
private IStream stream;
+ private MetaData metaData;
public HttpTransportOverHTTP2(Connector connector, HTTP2ServerConnection connection)
{
@@ -85,48 +88,60 @@ public class HttpTransportOverHTTP2 implements HttpTransport
public void send(MetaData.Response info, boolean isHeadRequest, ByteBuffer content, boolean lastContent, Callback callback)
{
boolean hasContent = BufferUtil.hasContent(content) && !isHeadRequest;
-
if (info != null)
{
+ metaData = info;
+
int status = info.getStatus();
boolean informational = HttpStatus.isInformational(status) && status != HttpStatus.SWITCHING_PROTOCOLS_101;
- boolean committed = false;
- if (!informational)
- committed = commit.compareAndSet(false, true);
-
- if (committed || informational)
+ if (informational)
{
- if (hasContent)
- {
- Callback commitCallback = new Callback.Nested(callback)
- {
- @Override
- public void succeeded()
- {
- if (transportCallback.start(callback, false))
- send(content, lastContent, transportCallback);
- }
- };
- if (transportCallback.start(commitCallback, true))
- commit(info, false, transportCallback);
- }
- else
- {
- if (transportCallback.start(callback, false))
- commit(info, lastContent, transportCallback);
- }
+ if (transportCallback.start(callback, false))
+ sendHeaders(info, false, transportCallback);
}
else
{
- callback.failed(new IllegalStateException("committed"));
+ boolean needsCommit = commit.compareAndSet(false, true);
+ if (needsCommit)
+ {
+ Supplier trailers = info.getTrailerSupplier();
+
+ if (hasContent)
+ {
+ Callback nested = trailers == null || !lastContent ? callback : new SendTrailers(callback);
+ Callback commitCallback = new Callback.Nested(nested)
+ {
+ @Override
+ public void succeeded()
+ {
+ if (transportCallback.start(nested, false))
+ sendContent(content, lastContent, trailers == null && lastContent, transportCallback);
+ }
+ };
+ if (transportCallback.start(commitCallback, true))
+ sendHeaders(info, false, transportCallback);
+ }
+ else
+ {
+ Callback nested = trailers == null ? callback : new SendTrailers(callback);
+ if (transportCallback.start(nested, true))
+ sendHeaders(info, trailers == null && lastContent, transportCallback);
+ }
+ }
+ else
+ {
+ callback.failed(new IllegalStateException("committed"));
+ }
}
}
else
{
if (hasContent || lastContent)
{
- if (transportCallback.start(callback, false))
- send(content, lastContent, transportCallback);
+ Supplier trailers = metaData.getTrailerSupplier();
+ Callback nested = trailers == null ? callback : new SendTrailers(callback);
+ if (transportCallback.start(nested, false))
+ sendContent(content, lastContent, trailers == null && lastContent, transportCallback);
}
else
{
@@ -171,7 +186,7 @@ public class HttpTransportOverHTTP2 implements HttpTransport
}, new Stream.Listener.Adapter()); // TODO: handle reset from the client ?
}
- private void commit(MetaData.Response info, boolean endStream, Callback callback)
+ private void sendHeaders(MetaData.Response info, boolean endStream, Callback callback)
{
if (LOG.isDebugEnabled())
{
@@ -185,7 +200,7 @@ public class HttpTransportOverHTTP2 implements HttpTransport
stream.headers(frame, callback);
}
- private void send(ByteBuffer content, boolean lastContent, Callback callback)
+ private void sendContent(ByteBuffer content, boolean lastContent, boolean endStream, Callback callback)
{
if (LOG.isDebugEnabled())
{
@@ -193,10 +208,22 @@ public class HttpTransportOverHTTP2 implements HttpTransport
stream.getId(), Integer.toHexString(stream.getSession().hashCode()),
content.remaining(), lastContent ? " (last chunk)" : "");
}
- DataFrame frame = new DataFrame(stream.getId(), content, lastContent);
+ DataFrame frame = new DataFrame(stream.getId(), content, endStream);
stream.data(frame, callback);
}
+ private void sendTrailers(MetaData metaData, Callback callback)
+ {
+ if (LOG.isDebugEnabled())
+ {
+ LOG.debug("HTTP2 Response #{}/{}: trailers",
+ stream.getId(), Integer.toHexString(stream.getSession().hashCode()));
+ }
+
+ HeadersFrame frame = new HeadersFrame(stream.getId(), metaData, null, true);
+ stream.headers(frame, callback);
+ }
+
public void onStreamFailure(Throwable failure)
{
transportCallback.failed(failure);
@@ -277,7 +304,9 @@ public class HttpTransportOverHTTP2 implements HttpTransport
}
}
if (LOG.isDebugEnabled())
- LOG.debug("HTTP2 Response #{} {}", stream.getId(), commit ? "committed" : "flushed content");
+ LOG.debug("HTTP2 Response #{}/{} {}",
+ stream.getId(), Integer.toHexString(stream.getSession().hashCode()),
+ commit ? "committed" : "flushed content");
if (callback != null)
callback.succeeded();
}
@@ -340,4 +369,19 @@ public class HttpTransportOverHTTP2 implements HttpTransport
{
IDLE, WRITING, FAILED, TIMEOUT
}
+
+ private class SendTrailers extends Callback.Nested
+ {
+ private SendTrailers(Callback callback)
+ {
+ super(callback);
+ }
+
+ @Override
+ public void succeeded()
+ {
+ if (transportCallback.start(getCallback(), false))
+ sendTrailers(new MetaData(HttpVersion.HTTP_2, metaData.getTrailerSupplier().get()), transportCallback);
+ }
+ }
}
diff --git a/jetty-server/src/main/java/org/eclipse/jetty/server/Response.java b/jetty-server/src/main/java/org/eclipse/jetty/server/Response.java
index 2a1f84a412f..2bb188c27d8 100644
--- a/jetty-server/src/main/java/org/eclipse/jetty/server/Response.java
+++ b/jetty-server/src/main/java/org/eclipse/jetty/server/Response.java
@@ -27,6 +27,7 @@ import java.util.EnumSet;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
import java.util.stream.Collectors;
import javax.servlet.RequestDispatcher;
@@ -107,6 +108,7 @@ public class Response implements HttpServletResponse
private OutputType _outputType = OutputType.NONE;
private ResponseWriter _writer;
private long _contentLength = -1;
+ private Supplier trailers;
private enum EncodingFrom { NOT_SET, INFERRED, SET_LOCALE, SET_CONTENT_TYPE, SET_CHARACTER_ENCODING }
private static final EnumSet __localeOverride = EnumSet.of(EncodingFrom.NOT_SET,EncodingFrom.INFERRED);
@@ -1308,10 +1310,20 @@ public class Response implements HttpServletResponse
_out.resetBuffer();
}
+ public void setTrailers(Supplier trailers)
+ {
+ this.trailers = trailers;
+ }
+
+ public Supplier getTrailers()
+ {
+ return trailers;
+ }
+
protected MetaData.Response newResponseMetaData()
{
MetaData.Response info = new MetaData.Response(_channel.getRequest().getHttpVersion(), getStatus(), getReason(), _fields, getLongContentLength());
- // TODO info.setTrailerSupplier(trailers);
+ info.setTrailerSupplier(getTrailers());
return info;
}
diff --git a/jetty-server/src/test/java/org/eclipse/jetty/server/HttpTrailersTest.java b/jetty-server/src/test/java/org/eclipse/jetty/server/HttpTrailersTest.java
deleted file mode 100644
index 2994717c2fe..00000000000
--- a/jetty-server/src/test/java/org/eclipse/jetty/server/HttpTrailersTest.java
+++ /dev/null
@@ -1,170 +0,0 @@
-//
-// ========================================================================
-// Copyright (c) 1995-2017 Mort Bay Consulting Pty. Ltd.
-// ------------------------------------------------------------------------
-// All rights reserved. This program and the accompanying materials
-// are made available under the terms of the Eclipse Public License v1.0
-// and Apache License v2.0 which accompanies this distribution.
-//
-// The Eclipse Public License is available at
-// http://www.eclipse.org/legal/epl-v10.html
-//
-// The Apache License v2.0 is available at
-// http://www.opensource.org/licenses/apache2.0.php
-//
-// You may elect to redistribute this code under either of these licenses.
-// ========================================================================
-//
-
-package org.eclipse.jetty.server;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.Socket;
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-
-import javax.servlet.ServletException;
-import javax.servlet.ServletInputStream;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import org.eclipse.jetty.http.HttpFields;
-import org.eclipse.jetty.http.HttpStatus;
-import org.eclipse.jetty.http.HttpTester;
-import org.eclipse.jetty.server.handler.AbstractHandler;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class HttpTrailersTest
-{
- private Server server;
- private ServerConnector connector;
-
- private void start(Handler handler) throws Exception
- {
- server = new Server();
- connector = new ServerConnector(server);
- server.addConnector(connector);
- server.setHandler(handler);
- server.start();
- }
-
- @After
- public void dispose() throws Exception
- {
- if (server != null)
- server.stop();
- }
-
- @Test
- public void testServletRequestTrailers() throws Exception
- {
- String trailerName = "Trailer";
- String trailerValue = "value";
- start(new AbstractHandler.ErrorDispatchHandler()
- {
- @Override
- protected void doNonErrorHandle(String target, Request jettyRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException
- {
- jettyRequest.setHandled(true);
-
- // Read the content first.
- ServletInputStream input = jettyRequest.getInputStream();
- while (true)
- {
- int read = input.read();
- if (read < 0)
- break;
- }
-
- // Now the trailers can be accessed.
- HttpFields trailers = jettyRequest.getTrailers();
- Assert.assertNotNull(trailers);
- Assert.assertEquals(trailerValue, trailers.get(trailerName));
- }
- });
-
- try (Socket client = new Socket("localhost", connector.getLocalPort()))
- {
- client.setSoTimeout(5000);
-
- String request = "" +
- "GET / HTTP/1.1\r\n" +
- "Host: localhost\r\n" +
- "Transfer-Encoding: chunked\r\n" +
- "\r\n" +
- "0\r\n" +
- trailerName + ": " + trailerValue + "\r\n" +
- "\r\n";
- OutputStream output = client.getOutputStream();
- output.write(request.getBytes(StandardCharsets.UTF_8));
- output.flush();
-
- HttpTester.Response response = HttpTester.parseResponse(HttpTester.from(client.getInputStream()));
- Assert.assertNotNull(response);
- Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
- }
- }
-
- @Test
- public void testHugeTrailer() throws Exception
- {
- start(new AbstractHandler.ErrorDispatchHandler()
- {
- @Override
- protected void doNonErrorHandle(String target, Request jettyRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException
- {
- jettyRequest.setHandled(true);
-
- try
- {
- // EOF will not be reached because of the huge trailer.
- ServletInputStream input = jettyRequest.getInputStream();
- while (true)
- {
- int read = input.read();
- if (read < 0)
- break;
- }
- Assert.fail();
- }
- catch (IOException x)
- {
- // Expected.
- }
- }
- });
-
- char[] huge = new char[1024 * 1024];
- Arrays.fill(huge, 'X');
- try (Socket client = new Socket("localhost", connector.getLocalPort()))
- {
- client.setSoTimeout(5000);
-
- try
- {
- String request = "" +
- "GET / HTTP/1.1\r\n" +
- "Host: localhost\r\n" +
- "Transfer-Encoding: chunked\r\n" +
- "\r\n" +
- "0\r\n" +
- "Trailer: " + new String(huge) + "\r\n" +
- "\r\n";
- OutputStream output = client.getOutputStream();
- output.write(request.getBytes(StandardCharsets.UTF_8));
- output.flush();
-
- HttpTester.Response response = HttpTester.parseResponse(HttpTester.from(client.getInputStream()));
- Assert.assertNotNull(response);
- Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
- }
- catch(Exception e)
- {
- // May be thrown if write fails and error handling is aborted
- }
- }
- }
-}
diff --git a/tests/test-http-client-transport/src/test/java/org/eclipse/jetty/http/client/HttpTrailersTest.java b/tests/test-http-client-transport/src/test/java/org/eclipse/jetty/http/client/HttpTrailersTest.java
new file mode 100644
index 00000000000..29e4ca5e3fb
--- /dev/null
+++ b/tests/test-http-client-transport/src/test/java/org/eclipse/jetty/http/client/HttpTrailersTest.java
@@ -0,0 +1,227 @@
+//
+// ========================================================================
+// Copyright (c) 1995-2017 Mort Bay Consulting Pty. Ltd.
+// ------------------------------------------------------------------------
+// All rights reserved. This program and the accompanying materials
+// are made available under the terms of the Eclipse Public License v1.0
+// and Apache License v2.0 which accompanies this distribution.
+//
+// The Eclipse Public License is available at
+// http://www.eclipse.org/legal/epl-v10.html
+//
+// The Apache License v2.0 is available at
+// http://www.opensource.org/licenses/apache2.0.php
+//
+// You may elect to redistribute this code under either of these licenses.
+// ========================================================================
+//
+
+package org.eclipse.jetty.http.client;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.servlet.ServletException;
+import javax.servlet.ServletInputStream;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.eclipse.jetty.client.HttpRequest;
+import org.eclipse.jetty.client.HttpResponse;
+import org.eclipse.jetty.client.api.ContentResponse;
+import org.eclipse.jetty.client.util.BytesContentProvider;
+import org.eclipse.jetty.http.HttpFields;
+import org.eclipse.jetty.http.HttpMethod;
+import org.eclipse.jetty.http.HttpStatus;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.Response;
+import org.eclipse.jetty.server.handler.AbstractHandler;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class HttpTrailersTest extends AbstractTest
+{
+ public HttpTrailersTest(Transport transport)
+ {
+ super(transport == Transport.FCGI ? null : transport);
+ }
+
+ @Test
+ public void testRequestTrailersNoContent() throws Exception
+ {
+ testRequestTrailers(null);
+ }
+
+ @Test
+ public void testRequestTrailersWithContent() throws Exception
+ {
+ testRequestTrailers("abcdefghijklmnopqrstuvwxyz".getBytes(StandardCharsets.UTF_8));
+ }
+
+ private void testRequestTrailers(byte[] content) throws Exception
+ {
+ String trailerName = "Trailer";
+ String trailerValue = "value";
+ start(new AbstractHandler.ErrorDispatchHandler()
+ {
+ @Override
+ protected void doNonErrorHandle(String target, Request jettyRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException
+ {
+ jettyRequest.setHandled(true);
+
+ // Read the content first.
+ ServletInputStream input = jettyRequest.getInputStream();
+ while (true)
+ {
+ int read = input.read();
+ if (read < 0)
+ break;
+ }
+
+ // Now the trailers can be accessed.
+ HttpFields trailers = jettyRequest.getTrailers();
+ Assert.assertNotNull(trailers);
+ Assert.assertEquals(trailerValue, trailers.get(trailerName));
+ }
+ });
+
+ HttpFields trailers = new HttpFields();
+ trailers.put(trailerName, trailerValue);
+
+ HttpRequest request = (HttpRequest)client.newRequest(newURI());
+ request = request.trailers(() -> trailers);
+ if (content != null)
+ request.method(HttpMethod.POST).content(new BytesContentProvider(content));
+ ContentResponse response = request.timeout(5, TimeUnit.SECONDS).send();
+ Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
+ }
+
+ @Test
+ public void testEmptyRequestTrailers() throws Exception
+ {
+ start(new AbstractHandler.ErrorDispatchHandler()
+ {
+ @Override
+ protected void doNonErrorHandle(String target, Request jettyRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException
+ {
+ jettyRequest.setHandled(true);
+
+ // Read the content first.
+ ServletInputStream input = jettyRequest.getInputStream();
+ while (true)
+ {
+ int read = input.read();
+ if (read < 0)
+ break;
+ }
+
+ // Now the trailers can be accessed.
+ HttpFields trailers = jettyRequest.getTrailers();
+ Assert.assertNull(trailers);
+ }
+ });
+
+ HttpFields trailers = new HttpFields();
+ HttpRequest request = (HttpRequest)client.newRequest(newURI());
+ request = request.trailers(() -> trailers);
+ ContentResponse response = request.timeout(5, TimeUnit.SECONDS).send();
+ Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
+ }
+
+ @Test
+ public void testResponseTrailersNoContent() throws Exception
+ {
+ testResponseTrailers(null);
+ }
+
+ @Test
+ public void testResponseTrailersWithContent() throws Exception
+ {
+ testResponseTrailers("abcdefghijklmnopqrstuvwxyz".getBytes(StandardCharsets.UTF_8));
+ }
+
+ private void testResponseTrailers(byte[] content) throws Exception
+ {
+ String trailerName = "Trailer";
+ String trailerValue = "value";
+ start(new AbstractHandler.ErrorDispatchHandler()
+ {
+ @Override
+ protected void doNonErrorHandle(String target, Request jettyRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException
+ {
+ jettyRequest.setHandled(true);
+
+ HttpFields trailers = new HttpFields();
+ trailers.put(trailerName, trailerValue);
+
+ Response jettyResponse = (Response)response;
+ jettyResponse.setTrailers(() -> trailers);
+ if (content != null)
+ response.getOutputStream().write(content);
+ }
+ });
+
+ AtomicReference failure = new AtomicReference<>(new Throwable("no_success"));
+ ContentResponse response = client.newRequest(newURI())
+ .onResponseSuccess(r ->
+ {
+ try
+ {
+ HttpResponse httpResponse = (HttpResponse)r;
+ HttpFields trailers = httpResponse.getTrailers();
+ Assert.assertNotNull(trailers);
+ Assert.assertEquals(trailerValue, trailers.get(trailerName));
+ failure.set(null);
+ }
+ catch (Throwable x)
+ {
+ failure.set(x);
+ }
+ })
+ .timeout(5, TimeUnit.SECONDS)
+ .send();
+ Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
+ Assert.assertNull(failure.get());
+ }
+
+ @Test
+ public void testEmptyResponseTrailers() throws Exception
+ {
+ start(new AbstractHandler.ErrorDispatchHandler()
+ {
+ @Override
+ protected void doNonErrorHandle(String target, Request jettyRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException
+ {
+ jettyRequest.setHandled(true);
+
+ HttpFields trailers = new HttpFields();
+
+ Response jettyResponse = (Response)response;
+ jettyResponse.setTrailers(() -> trailers);
+ }
+ });
+
+ AtomicReference failure = new AtomicReference<>(new Throwable("no_success"));
+ ContentResponse response = client.newRequest(newURI())
+ .onResponseSuccess(r ->
+ {
+ try
+ {
+ HttpResponse httpResponse = (HttpResponse)r;
+ HttpFields trailers = httpResponse.getTrailers();
+ Assert.assertNull(trailers);
+ failure.set(null);
+ }
+ catch (Throwable x)
+ {
+ failure.set(x);
+ }
+ })
+ .timeout(5, TimeUnit.SECONDS)
+ .send();
+ Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
+ Assert.assertNull(failure.get());
+ }
+}
From f53776628f15b22899026f05fed00b9610d06c8d Mon Sep 17 00:00:00 2001
From: Joakim Erdfelt
Date: Tue, 4 Apr 2017 09:08:44 -0700
Subject: [PATCH 12/16] Issue #1448 - Reduce unncessary URL creation
---
.../java/org/eclipse/jetty/util/TypeUtil.java | 4 +-
.../jetty/webapp/URLStreamHandlerUtil.java | 78 ++++++++++++
.../jetty/webapp/WebAppClassLoaderTest.java | 16 ++-
.../WebAppClassLoaderUrlStreamTest.java | 113 ++++++++++++++++++
4 files changed, 203 insertions(+), 8 deletions(-)
create mode 100644 jetty-webapp/src/test/java/org/eclipse/jetty/webapp/URLStreamHandlerUtil.java
create mode 100644 jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderUrlStreamTest.java
diff --git a/jetty-util/src/main/java/org/eclipse/jetty/util/TypeUtil.java b/jetty-util/src/main/java/org/eclipse/jetty/util/TypeUtil.java
index 8897375a763..69f76c79c0c 100644
--- a/jetty-util/src/main/java/org/eclipse/jetty/util/TypeUtil.java
+++ b/jetty-util/src/main/java/org/eclipse/jetty/util/TypeUtil.java
@@ -34,8 +34,6 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
-import javax.servlet.ServletContainerInitializer;
-
import org.eclipse.jetty.util.annotation.Name;
import org.eclipse.jetty.util.log.Log;
import org.eclipse.jetty.util.log.Logger;
@@ -723,7 +721,7 @@ public class TypeUtil
{
try
{
- return Resource.newResource(URIUtil.getJarSource(url.toString()));
+ return Resource.newResource(URIUtil.getJarSource(url.toURI()));
}
catch(Exception e)
{
diff --git a/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/URLStreamHandlerUtil.java b/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/URLStreamHandlerUtil.java
new file mode 100644
index 00000000000..1960a494c45
--- /dev/null
+++ b/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/URLStreamHandlerUtil.java
@@ -0,0 +1,78 @@
+//
+// ========================================================================
+// Copyright (c) 1995-2017 Mort Bay Consulting Pty. Ltd.
+// ------------------------------------------------------------------------
+// All rights reserved. This program and the accompanying materials
+// are made available under the terms of the Eclipse Public License v1.0
+// and Apache License v2.0 which accompanies this distribution.
+//
+// The Eclipse Public License is available at
+// http://www.eclipse.org/legal/epl-v10.html
+//
+// The Apache License v2.0 is available at
+// http://www.opensource.org/licenses/apache2.0.php
+//
+// You may elect to redistribute this code under either of these licenses.
+// ========================================================================
+//
+
+package org.eclipse.jetty.webapp;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.net.URL;
+import java.net.URLStreamHandlerFactory;
+import java.util.Arrays;
+import java.util.Optional;
+
+public final class URLStreamHandlerUtil
+{
+ public static void setFactory(URLStreamHandlerFactory factory)
+ {
+ try
+ {
+ // First, reset the factory field
+ Field factoryField = getURLStreamHandlerFactoryField();
+ factoryField.setAccessible(true);
+ factoryField.set(null, null);
+
+ if(factory != null)
+ {
+ // Next, set the factory
+ URL.setURLStreamHandlerFactory(factory);
+ }
+ }
+ catch(Throwable ignore)
+ {
+ ignore.printStackTrace(System.err);
+ }
+ }
+
+ public static URLStreamHandlerFactory getFactory()
+ {
+ try
+ {
+ // First, reset the factory field
+ Field factoryField = getURLStreamHandlerFactoryField();
+ factoryField.setAccessible(true);
+ return (URLStreamHandlerFactory) factoryField.get(null);
+ }
+ catch(Throwable ignore)
+ {
+ return null;
+ }
+ }
+
+ private static Field getURLStreamHandlerFactoryField()
+ {
+ Optional optFactoryField = Arrays.stream(URL.class.getDeclaredFields())
+ .filter((f) -> Modifier.isStatic(f.getModifiers()) &&
+ f.getType().equals(URLStreamHandlerFactory.class))
+ .findFirst();
+
+ if(optFactoryField.isPresent())
+ return optFactoryField.get();
+
+ throw new RuntimeException( "Cannot find URLStreamHandlerFactory field in " + URL.class.getName() );
+ }
+}
diff --git a/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderTest.java b/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderTest.java
index 8a1518a0086..2974ea9f3d0 100644
--- a/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderTest.java
+++ b/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderTest.java
@@ -18,15 +18,18 @@
package org.eclipse.jetty.webapp;
-import static org.eclipse.jetty.toolchain.test.ExtraMatchers.*;
-import static org.hamcrest.Matchers.*;
-import static org.junit.Assert.*;
+import static org.eclipse.jetty.toolchain.test.ExtraMatchers.ordered;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeThat;
import java.lang.instrument.ClassFileTransformer;
import java.lang.instrument.IllegalClassFormatException;
import java.net.URI;
import java.net.URL;
-import java.net.URLClassLoader;
import java.nio.file.Path;
import java.security.ProtectionDomain;
import java.util.ArrayList;
@@ -48,7 +51,7 @@ public class WebAppClassLoaderTest
private Path testWebappDir;
private WebAppContext _context;
- private WebAppClassLoader _loader;
+ protected WebAppClassLoader _loader;
@Before
public void init() throws Exception
@@ -241,6 +244,9 @@ public class WebAppClassLoaderTest
@Test
public void testResources() throws Exception
{
+ // The existence of a URLStreamHandler changes the behavior
+ assumeThat("No URLStreamHandler in place", URLStreamHandlerUtil.getFactory(), nullValue());
+
List expected = new ArrayList<>();
List resources;
diff --git a/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderUrlStreamTest.java b/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderUrlStreamTest.java
new file mode 100644
index 00000000000..296285052e6
--- /dev/null
+++ b/jetty-webapp/src/test/java/org/eclipse/jetty/webapp/WebAppClassLoaderUrlStreamTest.java
@@ -0,0 +1,113 @@
+//
+// ========================================================================
+// Copyright (c) 1995-2017 Mort Bay Consulting Pty. Ltd.
+// ------------------------------------------------------------------------
+// All rights reserved. This program and the accompanying materials
+// are made available under the terms of the Eclipse Public License v1.0
+// and Apache License v2.0 which accompanies this distribution.
+//
+// The Eclipse Public License is available at
+// http://www.eclipse.org/legal/epl-v10.html
+//
+// The Apache License v2.0 is available at
+// http://www.opensource.org/licenses/apache2.0.php
+//
+// You may elect to redistribute this code under either of these licenses.
+// ========================================================================
+//
+
+package org.eclipse.jetty.webapp;
+
+import java.net.URLStreamHandler;
+import java.net.URLStreamHandlerFactory;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.After;
+import org.junit.Before;
+
+public class WebAppClassLoaderUrlStreamTest extends WebAppClassLoaderTest
+{
+ public static class URLHandlers implements URLStreamHandlerFactory
+ {
+ private static final String[] STREAM_HANDLER_PREFIXES;
+
+ static
+ {
+ STREAM_HANDLER_PREFIXES = new String[]{
+ "sun.net.www.protocol",
+ "org.apache.harmony.luni.internal.net.www.protocol",
+ "javax.net.ssl"
+ };
+ }
+
+ private Map handlers = new HashMap<>();
+ private ClassLoader loader;
+
+ public URLHandlers(ClassLoader loader)
+ {
+ this.loader = loader;
+ }
+
+ private URLStreamHandler getBuiltInHandler(String protocol, ClassLoader classLoader)
+ {
+ URLStreamHandler handler = handlers.get(protocol);
+
+ if (handler == null)
+ {
+ for (String prefix : STREAM_HANDLER_PREFIXES)
+ {
+ String className = prefix + '.' + protocol + ".Handler";
+ try
+ {
+ Class> clazz = Class.forName(className, false, classLoader);
+ handler = (URLStreamHandler) clazz.newInstance();
+ break;
+ }
+ catch (Exception ignore)
+ {
+ ignore.printStackTrace(System.err);
+ }
+ }
+
+ if (handler != null)
+ {
+ handlers.put(protocol, handler);
+ }
+ }
+
+ if (handler == null)
+ {
+ throw new RuntimeException("Unable to find handler for protocol [" + protocol + "]");
+ }
+
+ return handler;
+ }
+
+ @Override
+ public URLStreamHandler createURLStreamHandler(String protocol)
+ {
+ try
+ {
+ return getBuiltInHandler(protocol, loader);
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException("Unable to create URLStreamHandler for protocol [" + protocol + "]");
+ }
+ }
+ }
+
+ @After
+ public void cleanupURLStreamHandlerFactory()
+ {
+ URLStreamHandlerUtil.setFactory(null);
+ }
+
+ @Before
+ public void init() throws Exception
+ {
+ super.init();
+ URLStreamHandlerUtil.setFactory(new URLHandlers(_loader));
+ }
+}
From 9cdf8b046810e6c7c971b55254ccf8c030667f54 Mon Sep 17 00:00:00 2001
From: Joakim Erdfelt
Date: Tue, 4 Apr 2017 09:18:37 -0700
Subject: [PATCH 13/16] Issue #1449 - Removing unused / broken __jvmLib
reference
---
.../eclipse/jetty/webapp/WebAppContext.java | 18 ------------------
1 file changed, 18 deletions(-)
diff --git a/jetty-webapp/src/main/java/org/eclipse/jetty/webapp/WebAppContext.java b/jetty-webapp/src/main/java/org/eclipse/jetty/webapp/WebAppContext.java
index c408952b08f..f8deb94bcbb 100644
--- a/jetty-webapp/src/main/java/org/eclipse/jetty/webapp/WebAppContext.java
+++ b/jetty-webapp/src/main/java/org/eclipse/jetty/webapp/WebAppContext.java
@@ -61,7 +61,6 @@ import org.eclipse.jetty.servlet.ServletHandler;
import org.eclipse.jetty.util.AttributesMap;
import org.eclipse.jetty.util.Loader;
import org.eclipse.jetty.util.MultiException;
-import org.eclipse.jetty.util.TypeUtil;
import org.eclipse.jetty.util.URIUtil;
import org.eclipse.jetty.util.annotation.ManagedAttribute;
import org.eclipse.jetty.util.annotation.ManagedObject;
@@ -130,23 +129,6 @@ public class WebAppContext extends ServletContextHandler implements WebAppClassL
"org.eclipse.jetty.servlets.PushSessionCacheFilter" //must be loaded by container classpath
} ;
- // Find the location of the JVM lib directory
- public final static String __jvmlib;
- static
- {
- String lib=null;
- try
- {
- lib=TypeUtil.getLoadedFrom(System.class).getFile().getParentFile().toURI().toString();
- }
- catch(Exception e)
- {
- LOG.warn(e);
- lib=null;
- }
- __jvmlib=lib;
- }
-
// Server classes are classes that are hidden from being
// loaded by the web application using system classloader,
// so if web application needs to load any of such classes,
From f90976f635d8ef0081ec25c27e41aec2acf13437 Mon Sep 17 00:00:00 2001
From: WalkerWatch
Date: Tue, 4 Apr 2017 16:25:03 -0400
Subject: [PATCH 14/16] Version reminder and misc cleanup.
Signed-off-by: WalkerWatch
---
.../configuring/security/authentication.adoc | 43 ++++++++++---------
.../getting-started/jetty-installing.adoc | 6 +++
.../introduction/jetty-coordinates.adoc | 8 +++-
.../introduction/what-version.adoc | 6 +++
4 files changed, 41 insertions(+), 22 deletions(-)
diff --git a/jetty-documentation/src/main/asciidoc/configuring/security/authentication.adoc b/jetty-documentation/src/main/asciidoc/configuring/security/authentication.adoc
index c58453c7ece..309c6e292d6 100644
--- a/jetty-documentation/src/main/asciidoc/configuring/security/authentication.adoc
+++ b/jetty-documentation/src/main/asciidoc/configuring/security/authentication.adoc
@@ -27,9 +27,9 @@ Authorization::
==== Configuring an Authentication mechanism
-The jetty server supports several standard authentication mechanisms: http://en.wikipedia.org/wiki/Basic_access_authentication[BASIC]; http://en.wikipedia.org/wiki/Digest_authentication[DIGEST]; http://en.wikipedia.org/wiki/Form-based_authentication[FORM]; CLIENT-CERT; and other mechanisms can be plugged in using the extensible http://docs.oracle.com/cd/E19462-01/819-6717/gcszc/index.html[JASPI] or http://en.wikipedia.org/wiki/SPNEGO[SPNEGO] mechanisms.
+Jetty server supports several standard authentication mechanisms: http://en.wikipedia.org/wiki/Basic_access_authentication[BASIC]; http://en.wikipedia.org/wiki/Digest_authentication[DIGEST]; http://en.wikipedia.org/wiki/Form-based_authentication[FORM]; CLIENT-CERT; and other mechanisms can be plugged in using the extensible http://docs.oracle.com/cd/E19462-01/819-6717/gcszc/index.html[JASPI] or http://en.wikipedia.org/wiki/SPNEGO[SPNEGO] mechanisms.
-Internally, configuring an authentication mechanism is done by setting an instance of a the link:{JDURL}/org/eclipse/jetty/security/Authenticator.html[Authenticator] interface onto the link:{JDURL}/org/eclipse/jetty/security/SecurityHandler.html[SecurityHandler] of the context, but in most cases it is done by declaring a `< login-config>` element in the standard web.xml descriptor or via annotations.
+Internally, configuring an authentication mechanism is done by setting an instance of a the link:{JDURL}/org/eclipse/jetty/security/Authenticator.html[Authenticator] interface onto the link:{JDURL}/org/eclipse/jetty/security/SecurityHandler.html[SecurityHandler] of the context, but in most cases it is done by declaring a `` element in the standard web.xml descriptor or via annotations.
Below is an example taken from the link:{GITBROWSEURL}/tests/test-webapps/test-jetty-webapp/src/main/webapp/WEB-INF/web.xml?h=release-9[jetty-test-webapp web.xml] that configures BASIC authentication:
@@ -103,7 +103,7 @@ When a request is received for a protected resource, the web container checks if
The Servlet Specification does not address how the static security information in the `WEB-INF/web.xml` file is mapped to the runtime environment of the container.
For Jetty, the link:{JDURL}/org/eclipse/jetty/security/LoginService.html[LoginService] performs this function.
-A LoginService has a unique name, and gives access to information about a set of users.
+A `LoginService` has a unique name, and gives access to information about a set of users.
Each user has authentication information (e.g. a password) and a set of roles associated with him/herself.
You may configure one or many different LoginServices depending on your needs.
@@ -114,7 +114,8 @@ When a request to a web application requires authentication or authorization, Je
==== Scoping Security Realms
-A LoginService has a unique name, and is composed of a set of users. Each user has authentication information (for example, a password) and a set of roles associated with him/herself.
+A `LoginService` has a unique name, and is composed of a set of users.
+Each user has authentication information (for example, a password) and a set of roles associated with him/herself.
You can configure one or many different realms depending on your needs.
* Configure a single LoginService to share common security information across all of your web applications.
@@ -144,8 +145,8 @@ Here's an example of an xml file that defines an in-memory type of LoginService
----
-If you define more than one LoginService on a Server, you will need to specify which one you want used for each context.
-You can do that by telling the context the name of the LoginService, or passing it the LoginService instance.
+If you define more than one `LoginService` on a Server, you will need to specify which one you want used for each context.
+You can do that by telling the context the name of the `LoginService`, or passing it the `LoginService` instance.
Here's an example of doing both of these, using a link:#deployable-descriptor-file[context xml file]:
[source, xml, subs="{sub-order}"]
@@ -170,7 +171,7 @@ Here's an example of doing both of these, using a link:#deployable-descriptor-fi
===== Per-Webapp Scoped
-Alternatively, you can define a LoginService for just a single web application.
+Alternatively, you can define a `LoginService` for just a single web application.
Here's how to define the same HashLoginService, but inside a link:#deployable-descriptor-file[context xml file]:
[source, xml, subs="{sub-order}"]
@@ -192,12 +193,12 @@ Here's how to define the same HashLoginService, but inside a link:#deployable-de
----
-Jetty provides a number of different LoginService types which can be seen in the next section.
+Jetty provides a number of different `LoginService` types which can be seen in the next section.
[[configuring-login-service]]
==== Configuring a LoginService
-A link:{JDURL}/org/eclipse/jetty/security/LoginService.html[LoginService] instance is required by each context/webapp that has a authentication mechanism, which is used to check the validity of the username and credentials collected by the authentication mechanism. Jetty provides the following implementations of LoginService:
+A link:{JDURL}/org/eclipse/jetty/security/LoginService.html[`LoginService`] instance is required by each context/webapp that has a authentication mechanism, which is used to check the validity of the username and credentials collected by the authentication mechanism. Jetty provides the following implementations of `LoginService`:
link:{JDURL}/org/eclipse/jetty/security/HashLoginService.html[HashLoginService]::
A user realm that is backed by a hash map that is filled either programatically or from a Java properties file.
@@ -211,16 +212,16 @@ link:{JDURL}/org/eclipse/jetty/jaas/JAASLoginService.html[JAASLoginService]::
link:{JDURL}/org/eclipse/jetty/security/SpnegoLoginService.html[SpnegoLoginService]::
http://en.wikipedia.org/wiki/SPNEGO[SPNEGO] Authentication; see the section on link:#spnego-support[SPNEGO support] for more information.
-An instance of a LoginService can be matched to a context/webapp by:
+An instance of a `LoginService` can be matched to a context/webapp by:
-* A LoginService instance may be set directly on the SecurityHandler instance via embedded code or IoC XML
-* Matching the realm-name defined in web.xml with the name of a LoginService instance that has been added to the Server instance as a dependent bean
-* If only a single LoginService instance has been set on the Server then it is used as the login service for the context
+* A `LoginService` instance may be set directly on the `SecurityHandler` instance via embedded code or IoC XML
+* Matching the realm-name defined in web.xml with the name of a `LoginService` instance that has been added to the Server instance as a dependent bean
+* If only a single `LoginService` instance has been set on the Server then it is used as the login service for the context
[[hash-login-service]]
===== HashLoginService
-The HashLoginService is a simple and efficient login service that loads usernames, credentials and roles from a Java properties file in the format:
+The `HashLoginService` is a simple and efficient login service that loads usernames, credentials and roles from a Java properties file in the format:
[source,properties]
----
@@ -249,7 +250,7 @@ guest: guest,read-only
----
-You configure the HashLoginService with a name and a reference to the location of the properties file:
+You configure the `HashLoginService` with a name and a reference to the location of the properties file:
[source, xml, subs="{sub-order}"]
----
@@ -378,12 +379,12 @@ When a user requests a resource that is access protected, the LoginService will
Until Servlet 3.1, role-based authorization could define:
-* access granted to a set of named roles
-* access totally forbidden, regardless of role
-* access granted to a user in any of the roles defined in the effective web.xml.
-This is indicated by the special value of "*" for the `` of a ` `in the ``
+* Access granted to a set of named roles
+* Access totally forbidden, regardless of role
+* Access granted to a user in any of the roles defined in the effective web.xml.
+This is indicated by the special value of `*` for the `` of a `` in the ``
With the advent of Servlet 3.1, there is now another authorization:
-* access granted to any user who is authenticated, regardless of roles.
-This is indicated by the special value of "**" for the `` of a `` in the ``
+* Access granted to any user who is authenticated, regardless of roles.
+This is indicated by the special value of `**` for the `` of a `` in the ``
diff --git a/jetty-documentation/src/main/asciidoc/quick-start/getting-started/jetty-installing.adoc b/jetty-documentation/src/main/asciidoc/quick-start/getting-started/jetty-installing.adoc
index e66716fb3a5..4069c53aec9 100644
--- a/jetty-documentation/src/main/asciidoc/quick-start/getting-started/jetty-installing.adoc
+++ b/jetty-documentation/src/main/asciidoc/quick-start/getting-started/jetty-installing.adoc
@@ -30,6 +30,12 @@ When you download and unpack the binary, it is extracted into a directory called
Put this directory in a convenient location.
The rest of the instructions in this documentation refer to this location as either `JETTY_HOME` or as `$(jetty.home).`
+_____
+[IMPORTANT]
+It is important that only stable releases are used in production environments.
+Versions that have been deprecated or are released as Milestones (M) or Release Candidates (RC) are not suitable for production as they may contain security flaws or incomplete/non-functioning feature sets.
+_____
+
[[distribution-content]]
==== Distribution Content
diff --git a/jetty-documentation/src/main/asciidoc/quick-start/introduction/jetty-coordinates.adoc b/jetty-documentation/src/main/asciidoc/quick-start/introduction/jetty-coordinates.adoc
index 5570f184cc3..8350bf18633 100644
--- a/jetty-documentation/src/main/asciidoc/quick-start/introduction/jetty-coordinates.adoc
+++ b/jetty-documentation/src/main/asciidoc/quick-start/introduction/jetty-coordinates.adoc
@@ -17,6 +17,12 @@
[[quickstart-jetty-coordinates]]
=== Finding Jetty in Maven
+_____
+[IMPORTANT]
+It is important that only stable releases are used in production environments.
+Versions that have been deprecated or are released as Milestones (M) or Release Candidates (RC) are not suitable for production as they may contain security flaws or incomplete/non-functioning feature sets.
+_____
+
==== Maven Coordinates
Jetty has existed in Maven Central almost since its inception, though the coordinates have changed over the years.
@@ -34,7 +40,7 @@ The top level Project Object Model (POM) for the Jetty project is located under
----
-==== Changelogs in Central
+==== Changelogs in Maven Central
The changes between versions of Jetty are tracked in a file called VERSIONS.txt, which is under source control and is generated on release.
Those generated files are also uploaded into Maven Central during the release of the top level POM. You can find them as a classifier marked artifact.
diff --git a/jetty-documentation/src/main/asciidoc/quick-start/introduction/what-version.adoc b/jetty-documentation/src/main/asciidoc/quick-start/introduction/what-version.adoc
index 93100663081..ed8672a4247 100644
--- a/jetty-documentation/src/main/asciidoc/quick-start/introduction/what-version.adoc
+++ b/jetty-documentation/src/main/asciidoc/quick-start/introduction/what-version.adoc
@@ -21,6 +21,12 @@ Jetty 9 is the most recent version of Jetty and has a great many improvements ov
This documentation which focuses on Jetty 9.
While many people continue to use older versions of Jetty, we generally recommend using Jetty 9 as it represents the version of Jetty that we will actively maintain and improve over the next few years.
+_____
+[IMPORTANT]
+It is important that only stable releases are used in production environments.
+Versions that have been deprecated or are released as Milestones (M) or Release Candidates (RC) are not suitable for production as they may contain security flaws or incomplete/non-functioning feature sets.
+_____
+
.Jetty Versions
[width="100%",cols="12%,9%,15%,6%,21%,10%,6%,21%",options="header",]
|=======================================================================
From 7fc3dac5f340e6bc053240c084e82cdea4274bca Mon Sep 17 00:00:00 2001
From: Simone Bordet
Date: Tue, 4 Apr 2017 14:41:10 +0200
Subject: [PATCH 15/16] Removed truststore.jks, not needed.
---
.../client/AbstractHttpClientServerTest.java | 2 --
.../eclipse/jetty/client/HttpClientTLSTest.java | 2 --
.../client/TLSServerConnectionCloseTest.java | 2 --
jetty-client/src/test/resources/truststore.jks | Bin 916 -> 0 bytes
4 files changed, 6 deletions(-)
delete mode 100644 jetty-client/src/test/resources/truststore.jks
diff --git a/jetty-client/src/test/java/org/eclipse/jetty/client/AbstractHttpClientServerTest.java b/jetty-client/src/test/java/org/eclipse/jetty/client/AbstractHttpClientServerTest.java
index c6fdaffdacb..b591cd7481f 100644
--- a/jetty-client/src/test/java/org/eclipse/jetty/client/AbstractHttpClientServerTest.java
+++ b/jetty-client/src/test/java/org/eclipse/jetty/client/AbstractHttpClientServerTest.java
@@ -71,8 +71,6 @@ public abstract class AbstractHttpClientServerTest
sslContextFactory.setEndpointIdentificationAlgorithm("");
sslContextFactory.setKeyStorePath("src/test/resources/keystore.jks");
sslContextFactory.setKeyStorePassword("storepwd");
- sslContextFactory.setTrustStorePath("src/test/resources/truststore.jks");
- sslContextFactory.setTrustStorePassword("storepwd");
}
if (server == null)
diff --git a/jetty-client/src/test/java/org/eclipse/jetty/client/HttpClientTLSTest.java b/jetty-client/src/test/java/org/eclipse/jetty/client/HttpClientTLSTest.java
index 5b1e53a3402..70e78b54fab 100644
--- a/jetty-client/src/test/java/org/eclipse/jetty/client/HttpClientTLSTest.java
+++ b/jetty-client/src/test/java/org/eclipse/jetty/client/HttpClientTLSTest.java
@@ -72,8 +72,6 @@ public class HttpClientTLSTest
sslContextFactory.setEndpointIdentificationAlgorithm("");
sslContextFactory.setKeyStorePath("src/test/resources/keystore.jks");
sslContextFactory.setKeyStorePassword("storepwd");
- sslContextFactory.setTrustStorePath("src/test/resources/truststore.jks");
- sslContextFactory.setTrustStorePassword("storepwd");
return sslContextFactory;
}
diff --git a/jetty-client/src/test/java/org/eclipse/jetty/client/TLSServerConnectionCloseTest.java b/jetty-client/src/test/java/org/eclipse/jetty/client/TLSServerConnectionCloseTest.java
index 929eb2faaf1..11437de7539 100644
--- a/jetty-client/src/test/java/org/eclipse/jetty/client/TLSServerConnectionCloseTest.java
+++ b/jetty-client/src/test/java/org/eclipse/jetty/client/TLSServerConnectionCloseTest.java
@@ -69,8 +69,6 @@ public class TLSServerConnectionCloseTest
sslContextFactory.setEndpointIdentificationAlgorithm("");
sslContextFactory.setKeyStorePath("src/test/resources/keystore.jks");
sslContextFactory.setKeyStorePassword("storepwd");
- sslContextFactory.setTrustStorePath("src/test/resources/truststore.jks");
- sslContextFactory.setTrustStorePassword("storepwd");
QueuedThreadPool clientThreads = new QueuedThreadPool();
clientThreads.setName("client");
diff --git a/jetty-client/src/test/resources/truststore.jks b/jetty-client/src/test/resources/truststore.jks
deleted file mode 100644
index 839cb8c35151c2b7c64afca24b6b72caad070a05..0000000000000000000000000000000000000000
GIT binary patch
literal 0
HcmV?d00001
literal 916
zcmezO_TO6u1_mY|W(3o$xs}
zE~X|%Muz1J{3AIFGbaABoD&*5saD@gH|APIn|qhRGl}gsUzm=o9G*UXZaLfkb^*)o
zjA*-gTf)`m_MQJYE&gJ}p^PHkrj!4^W|XX5a=N7A{;n#yaON&k_bHloe-^*hm?Z91
zlB>xeD=<(C>yn{9D54u}krkl}HQ(Uscha(++qf!T9y+xaEfnXd1O
zi0)T?voO%;QH9LK;*_O3mBblqm)!31vU@hm;^%>mh5U@y3R%l0gzi`2yxH!+?kPOi
zt!Tnsz1x9B3U2~8STZp)GB6^C5HPs_Lx_=~O<3xi>MmQ;D_g$D<_pdct`+TyzWTQ=
zW5Finm(sGEe;ty^>vg$!cV)t>;H#Mev23$*WWBpyJ}Ir;RW+Htrt6{Pk&qz&-XG2@
z8@{&Lu%DX7m47Uny+-3w`=4V611q#Ub(U`xZCtSK^2LO^3(s|HW&N14dV4@A&(kX%
z*S_eUPs-bSWRp>avt;CP@7K+G&3=b&1eO-s3f`;Cf91p#$)FW&xME3L8sEBQQDVCvfG>mdwqnk+GXd2ihXqpv
z;usF(WoYYmu8DZZa4%1z=+hI+*gpkUykAy5tj#grb*gH!M6TqIcifYBGVe^&T#-2O
K*=+x>r_BKeJV|!|
From e7ac447d63bfd4b9f7c15f63eeefd411c3f4e9c9 Mon Sep 17 00:00:00 2001
From: Simone Bordet
Date: Wed, 5 Apr 2017 15:27:43 +0200
Subject: [PATCH 16/16] Fixes #1452 - Add tests for [want|need]ClientAuth.
---
.../client/ssl/NeedWantClientAuthTest.java | 233 ++++++++++++++++++
.../src/test/resources/client_keystore.jks | Bin 0 -> 3956 bytes
2 files changed, 233 insertions(+)
create mode 100644 jetty-client/src/test/java/org/eclipse/jetty/client/ssl/NeedWantClientAuthTest.java
create mode 100644 jetty-client/src/test/resources/client_keystore.jks
diff --git a/jetty-client/src/test/java/org/eclipse/jetty/client/ssl/NeedWantClientAuthTest.java b/jetty-client/src/test/java/org/eclipse/jetty/client/ssl/NeedWantClientAuthTest.java
new file mode 100644
index 00000000000..e849564df73
--- /dev/null
+++ b/jetty-client/src/test/java/org/eclipse/jetty/client/ssl/NeedWantClientAuthTest.java
@@ -0,0 +1,233 @@
+//
+// ========================================================================
+// Copyright (c) 1995-2017 Mort Bay Consulting Pty. Ltd.
+// ------------------------------------------------------------------------
+// All rights reserved. This program and the accompanying materials
+// are made available under the terms of the Eclipse Public License v1.0
+// and Apache License v2.0 which accompanies this distribution.
+//
+// The Eclipse Public License is available at
+// http://www.eclipse.org/legal/epl-v10.html
+//
+// The Apache License v2.0 is available at
+// http://www.opensource.org/licenses/apache2.0.php
+//
+// You may elect to redistribute this code under either of these licenses.
+// ========================================================================
+//
+
+package org.eclipse.jetty.client.ssl;
+
+import java.security.cert.Certificate;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import javax.net.ssl.SSLHandshakeException;
+import javax.net.ssl.SSLSession;
+
+import org.eclipse.jetty.client.EmptyServerHandler;
+import org.eclipse.jetty.client.HttpClient;
+import org.eclipse.jetty.client.api.ContentResponse;
+import org.eclipse.jetty.http.HttpStatus;
+import org.eclipse.jetty.io.ssl.SslHandshakeListener;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
+import org.hamcrest.Matchers;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * In order to work, client authentication needs a certificate
+ * signed by a CA that also signed the server certificate.
+ *
+ * For this test, the client certificate is signed with the server
+ * certificate, and the server certificate is self-signed.
+ */
+public class NeedWantClientAuthTest
+{
+ private Server server;
+ private ServerConnector connector;
+ private HttpClient client;
+
+ private void startServer(SslContextFactory sslContextFactory, Handler handler) throws Exception
+ {
+ QueuedThreadPool serverThreads = new QueuedThreadPool();
+ serverThreads.setName("server");
+ server = new Server(serverThreads);
+
+ connector = new ServerConnector(server, sslContextFactory);
+ server.addConnector(connector);
+
+ server.setHandler(handler);
+ server.start();
+ }
+
+ private void startClient(SslContextFactory sslContextFactory) throws Exception
+ {
+ QueuedThreadPool clientThreads = new QueuedThreadPool();
+ clientThreads.setName("client");
+ client = new HttpClient(sslContextFactory);
+ client.setExecutor(clientThreads);
+ client.start();
+ }
+
+ private SslContextFactory createSslContextFactory()
+ {
+ SslContextFactory sslContextFactory = new SslContextFactory();
+ sslContextFactory.setEndpointIdentificationAlgorithm("");
+ sslContextFactory.setKeyStorePath("src/test/resources/keystore.jks");
+ sslContextFactory.setKeyStorePassword("storepwd");
+ return sslContextFactory;
+ }
+
+ @After
+ public void dispose() throws Exception
+ {
+ if (client != null)
+ client.stop();
+ if (server != null)
+ server.stop();
+ }
+
+ @Test
+ public void testWantClientAuthWithoutAuth() throws Exception
+ {
+ SslContextFactory serverSSL = new SslContextFactory();
+ serverSSL.setKeyStorePath("src/test/resources/keystore.jks");
+ serverSSL.setKeyStorePassword("storepwd");
+ serverSSL.setWantClientAuth(true);
+ startServer(serverSSL, new EmptyServerHandler());
+
+ SslContextFactory clientSSL = new SslContextFactory(true);
+ startClient(clientSSL);
+
+ ContentResponse response = client.newRequest("https://localhost:" + connector.getLocalPort())
+ .timeout(5, TimeUnit.SECONDS)
+ .send();
+
+ Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
+ }
+
+ @Test
+ public void testWantClientAuthWithAuth() throws Exception
+ {
+ SslContextFactory serverSSL = new SslContextFactory();
+ serverSSL.setKeyStorePath("src/test/resources/keystore.jks");
+ serverSSL.setKeyStorePassword("storepwd");
+ serverSSL.setWantClientAuth(true);
+ startServer(serverSSL, new EmptyServerHandler());
+ CountDownLatch handshakeLatch = new CountDownLatch(1);
+ connector.addBean(new SslHandshakeListener()
+ {
+ @Override
+ public void handshakeSucceeded(Event event)
+ {
+ try
+ {
+ SSLSession session = event.getSSLEngine().getSession();
+ Certificate[] clientCerts = session.getPeerCertificates();
+ Assert.assertNotNull(clientCerts);
+ Assert.assertThat(clientCerts.length, Matchers.greaterThan(0));
+ handshakeLatch.countDown();
+ }
+ catch (Throwable x)
+ {
+ x.printStackTrace();
+ }
+ }
+ });
+
+ SslContextFactory clientSSL = new SslContextFactory(true);
+ clientSSL.setKeyStorePath("src/test/resources/client_keystore.jks");
+ clientSSL.setKeyStorePassword("storepwd");
+ startClient(clientSSL);
+
+ ContentResponse response = client.newRequest("https://localhost:" + connector.getLocalPort())
+ .timeout(5, TimeUnit.SECONDS)
+ .send();
+
+ Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
+ Assert.assertTrue(handshakeLatch.await(5, TimeUnit.SECONDS));
+ }
+
+ @Test
+ public void testNeedClientAuthWithoutAuth() throws Exception
+ {
+ SslContextFactory serverSSL = new SslContextFactory();
+ serverSSL.setKeyStorePath("src/test/resources/keystore.jks");
+ serverSSL.setKeyStorePassword("storepwd");
+ serverSSL.setNeedClientAuth(true);
+ startServer(serverSSL, new EmptyServerHandler());
+
+ SslContextFactory clientSSL = new SslContextFactory(true);
+ startClient(clientSSL);
+ CountDownLatch handshakeLatch = new CountDownLatch(1);
+ client.addBean(new SslHandshakeListener()
+ {
+ @Override
+ public void handshakeFailed(Event event, Throwable failure)
+ {
+ Assert.assertThat(failure, Matchers.instanceOf(SSLHandshakeException.class));
+ handshakeLatch.countDown();
+ }
+ });
+
+ CountDownLatch latch = new CountDownLatch(1);
+ client.newRequest("https://localhost:" + connector.getLocalPort())
+ .timeout(5, TimeUnit.SECONDS)
+ .send(result ->
+ {
+ if (result.isFailed())
+ latch.countDown();
+ });
+
+ Assert.assertTrue(handshakeLatch.await(5, TimeUnit.SECONDS));
+ Assert.assertTrue(latch.await(5, TimeUnit.SECONDS));
+ }
+
+ @Test
+ public void testNeedClientAuthWithAuth() throws Exception
+ {
+ SslContextFactory serverSSL = new SslContextFactory();
+ serverSSL.setKeyStorePath("src/test/resources/keystore.jks");
+ serverSSL.setKeyStorePassword("storepwd");
+ serverSSL.setNeedClientAuth(true);
+ startServer(serverSSL, new EmptyServerHandler());
+ CountDownLatch handshakeLatch = new CountDownLatch(1);
+ connector.addBean(new SslHandshakeListener()
+ {
+ @Override
+ public void handshakeSucceeded(Event event)
+ {
+ try
+ {
+ SSLSession session = event.getSSLEngine().getSession();
+ Certificate[] clientCerts = session.getPeerCertificates();
+ Assert.assertNotNull(clientCerts);
+ Assert.assertThat(clientCerts.length, Matchers.greaterThan(0));
+ handshakeLatch.countDown();
+ }
+ catch (Throwable x)
+ {
+ x.printStackTrace();
+ }
+ }
+ });
+
+ SslContextFactory clientSSL = new SslContextFactory(true);
+ clientSSL.setKeyStorePath("src/test/resources/client_keystore.jks");
+ clientSSL.setKeyStorePassword("storepwd");
+ startClient(clientSSL);
+
+ ContentResponse response = client.newRequest("https://localhost:" + connector.getLocalPort())
+ .timeout(5, TimeUnit.SECONDS)
+ .send();
+
+ Assert.assertEquals(HttpStatus.OK_200, response.getStatus());
+ Assert.assertTrue(handshakeLatch.await(5, TimeUnit.SECONDS));
+ }
+}
diff --git a/jetty-client/src/test/resources/client_keystore.jks b/jetty-client/src/test/resources/client_keystore.jks
new file mode 100644
index 0000000000000000000000000000000000000000..9c31ff30c63b5efe401234d044db32cea9e157df
GIT binary patch
literal 3956
zcmeI#c{tSj9tZGQFxhF!l07O*G&9z5kSV*7QI@eC$ymlR!%#w!!7#QYOUO-G#!?ZY
zl#pezH`W+JqQ*s(y>QiYPS1JnxzD}#uk+XW_0R`Ed`+czw3ObE~?}rI|zXO4CVqATR(kR~`B3uB*_dVl+LQDxk
zuDNMlGuZ+EvKn6xl2907wPG~pTubr{@1
zgdYL_9>7$83;xolAp;NpaTq{$0GJHq2f)ZcE)W?A1kmy9&pU_rz
z8p2YH<6*NRwK^*xx?FQ+W^ORYZma+@!OKzaAJuk8bCY|QGh7(hmtG`ga{JDzSI~ok
z&Y;T^b)k}W;COZ8GzGnH`3v%?!e&8!N<2{}CWf+jiCx`f|Iv;z%P0&_u9ddnRfp{U
zu}xd^lWFk7n)XZSTLEK**~#<>O9HtDNKlGa&}5kXf~~$D+!xuYQEjM-u|F#!&ST`H
zeCUxu@AIg}oLt>leh~<0xDM-h{mA@XE5y6Y;*3CC;nTyy!0-ZQyqVr?w$^52++BX)
zPyqX-6q^P)&XHJsh$%GJh?Fv}!n7hu&s$X>L0}*N_{RzD9hbcc0Ra@~p|vA*y6OYf
z>&`^MThQ=>@s5MrSH1EoWm#epRa-B`2}uTw@sjg%=LYct&KxB!(BwmCws$oIv5YiqK)tVBIddwei5?_odV{P^9T1V!(S
zw6a>vMV^^~xS1(p30_Ld)ce>)z*~Ym>GOM}^EZEO-j&j+a9`EBj?gNRz0v^z?0z9W?jb)q
zm!Wq90AL5g$ZXqt<=|4{0I~rUL4tez0pCj-;G*CPdroFB&z%FHMA@%%*JF~|Vg%
zdU`8MT&C|3wTt2swF_>>T>6SpHL$|s$!eyMz@?3W^!C>SsO6)>o7Ab~n1@puThXG#1lFOq
z`}|tf)_RoXR~J?j`HXO+7c(Nz2xt0jSi3@)
z-z16iiz~1y*6qRk^|~p@_4|RH8rf(DMgPdvY?47m!61X0Jed*w%DTzcBLuzF5@u4h
z8G5X(fU!QX?uNWDcA23b}5@-`$3F>{gcJ)k!JmJ^pCy@{ACxh1}1RHKH
zn7v9E^L5oPlYE#j`%IteB+gLl0=*Ba(Q_Amn
z{`7RljUiF(Lx>W=Vgn5+m$c9{m6)5AQIM#zGTh_+$413CD%*{o)fcBpO^J7d6x<)u
zL!vwn0#X*QX(MxGhSd`Jb8M>~4K>I!(PwrP7b8TVZ#J%a+^tV}D5Z<~dfqnLaTf{W
z#n9rv*-03A;@VC$_#wr2(Iz7aca723X1XrwCda9$ykls!-G3PJg(-M-OmHIE-2wh@
zb@;4{Qb}33#Jm=^)t+yeO*7BXv+upmGU$9IrZnKEn^4ch?z&(%U6&MpaTq8akX&w~
zUE2C|TsEM?Ty}UV0}EHi2VSKt>&shRk__}&$c^8w$kWk!8LQo150ZK*{MKmWmTgv-
ze}BX;;}0$})$=HN+v-Pd=eFmV^6l5D0{H^5R-^cN&FK+W)pY;kC#2Vi+V`)XGwRq;
zC+o3f8fF4$Gua;;6}5_8I_)3NH8pvQyXn(BC4k-BLFJ)B?A7?s+N<5;>GZGlKBkU~
zBswLj-z1Qg${H(C?P4x?95;B=U)!Y|U@B_-r5>xtCMN$b96tcTi^OE^5FMS%5k>dM
zFp&pmWw&TkcRyr(ednL0eCXqovQ;}|-fVg2{%xAKgzs!Xsb9;*l}2x$Y^DXa
zBU(u8!?Mupov*om0{Uf{n0!y$krZw!?HUOJ&hHv7Xr%b+YF51{KRakqHhF{G#W*^q
z$c&ErG&^0#-Y-{3-Na~3Mk(4nbImyTlv#DBX=!WU5EC7Or;RBrrSeSN6Kj^Fh}-X9
zEp&o+p5Cs%b@SDe2Hb|E=ng3x{kjh8G$G|~pdor;QTAqKYA?qRjQqrh;~qZj_V95M
z#0hAH47e(YjMxA1Yr_2(<3n-}ADS>Vm>OIip$0=b2=gMoFA={jVFAC>h64(*A@~r8
z!GzzFeSb>i$b(t(C|uHOe-9_Fkp1nm%@6
zLs+pYUexDG);O<8WP_=%Cb23U_jLJELm`R8@XVLQcG8_?<1I#8gCo9hztDRUA!s~h
z*kY2h`)DCc44CQCU5}sSV@o{XPPo`?gnMD3H3RM_xb`&p1Hzk>B#^x`^FttHy`miX
z@G9x_2?OF(Uo&QS+v2EaEHiptYc1@~3n!-)KVIds%=|WIna`&hRVycNXRx5GLf-@x
zx^$$BjgBe#4Bxs~7JvSm7Cvi&_kJF}w%u4G0ajJ$2|R9%sTUZS}#zvVAh;dWNmU
zZnH&Y1<)Q1K0a|&%D^~=5;i7F?NE5rmrwi)i0@K;
literal 0
HcmV?d00001