Improve handling of SSL metadata in ConnectionMetaData, proxied connections and forwarded requests #10775 (#10853)

Improve handling of SSL metadata in ConnectionMetaData, proxied connections and forwarded requests

+ Added an Attributes.Synthetic mechanism to better (and mor commonly) handle additional attributes, whilst meeting the new TCK requirement that they can be deleted or replaced.
+ Made SslSessionData a reusable type provided by EndPoint
+ Added the EndPoint.Securable subtype that can provide a SslSessionData instance for secure EndPoints
+ Updated ForwardedRequestCustomizer, SecureRequestCustomizer to understand Securable and SslSessionData
+ Javadocs improvements.
+ Removed unnecessary attribute constants in SecureRequestCustomizer.

Signed-off-by: Simone Bordet <simone.bordet@gmail.com>

---------

Signed-off-by: Olivier Lamy <olamy@apache.org>
Signed-off-by: Simone Bordet <simone.bordet@gmail.com>
Co-authored-by: Olivier Lamy <olamy@apache.org>
Co-authored-by: Simone Bordet <simone.bordet@gmail.com>
This commit is contained in:
Greg Wilkins 2023-11-09 14:07:19 +11:00 committed by GitHub
parent 236061ea29
commit 6edd6d0ed0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
47 changed files with 1100 additions and 962 deletions

View File

@ -54,7 +54,7 @@ public class ConscryptHTTP2ClientTest
Assumptions.assumeTrue(canConnectTo(host, port));
Security.insertProviderAt(new OpenSSLProvider(), 1);
SslContextFactory sslContextFactory = new SslContextFactory.Client();
SslContextFactory.Client sslContextFactory = new SslContextFactory.Client();
sslContextFactory.setProvider("Conscrypt");
Conscrypt.setDefaultHostnameVerifier((certs, hostname, session) -> true);

View File

@ -48,7 +48,7 @@ public class JDK9HTTP2ClientTest
HTTP2Client client = new HTTP2Client();
try
{
SslContextFactory sslContextFactory = new SslContextFactory.Client();
SslContextFactory.Client sslContextFactory = new SslContextFactory.Client();
client.addBean(sslContextFactory);
client.start();

View File

@ -186,7 +186,7 @@ public class HttpClientProxyProtocolTest
if (Request.getPathInContext(request).equals("/tls_version"))
{
assertNotNull(proxyEndPoint.getTLV(typeTLS));
assertEquals(tlsVersion, proxyEndPoint.getAttribute(ProxyConnectionFactory.TLS_VERSION));
assertNotNull(proxyEndPoint.getSslSessionData());
}
response.getHeaders().put(HttpHeader.CONTENT_TYPE, MimeTypes.Type.TEXT_PLAIN.asString());
Content.Sink.write(response, true, String.valueOf(Request.getRemotePort(request)), callback);

View File

@ -27,7 +27,6 @@ import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
@ -632,7 +631,7 @@ public class HttpClientTLSTest
@Override
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected int networkFill(ByteBuffer input) throws IOException
@ -669,9 +668,9 @@ public class HttpClientTLSTest
return new SslClientConnectionFactory(sslContextFactory, getByteBufferPool(), getExecutor(), connectionFactory)
{
@Override
protected SslConnection newSslConnection(ByteBufferPool bufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(bufferPool, executor, endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected int networkFill(ByteBuffer input) throws IOException
@ -724,7 +723,7 @@ public class HttpClientTLSTest
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
ByteBufferPool bufferPool = connector.getByteBufferPool();
return new SslConnection(bufferPool, connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(bufferPool, connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected int networkFill(ByteBuffer input) throws IOException
@ -801,7 +800,7 @@ public class HttpClientTLSTest
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
ByteBufferPool bufferPool = connector.getByteBufferPool();
return new SslConnection(bufferPool, connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(bufferPool, connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected boolean networkFlush(ByteBuffer output) throws IOException
@ -872,7 +871,7 @@ public class HttpClientTLSTest
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
ByteBufferPool bufferPool = connector.getByteBufferPool();
return new SslConnection(bufferPool, connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(bufferPool, connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected boolean networkFlush(ByteBuffer output) throws IOException
@ -957,7 +956,7 @@ public class HttpClientTLSTest
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
ByteBufferPool bufferPool = connector.getByteBufferPool();
return new SslConnection(bufferPool, connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(bufferPool, connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected boolean networkFlush(ByteBuffer output) throws IOException
@ -1019,7 +1018,7 @@ public class HttpClientTLSTest
@Override
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected int networkFill(ByteBuffer input) throws IOException
@ -1057,9 +1056,9 @@ public class HttpClientTLSTest
return new SslClientConnectionFactory(sslContextFactory, getByteBufferPool(), getExecutor(), connectionFactory)
{
@Override
protected SslConnection newSslConnection(ByteBufferPool bufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(bufferPool, executor, endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected int networkFill(ByteBuffer input) throws IOException
@ -1119,9 +1118,9 @@ public class HttpClientTLSTest
return new SslClientConnectionFactory(sslContextFactory, getByteBufferPool(), getExecutor(), connectionFactory)
{
@Override
protected SslConnection newSslConnection(ByteBufferPool bufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(bufferPool, executor, endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected SSLEngineResult wrap(SSLEngine sslEngine, ByteBuffer[] input, ByteBuffer output) throws SSLException
@ -1160,7 +1159,7 @@ public class HttpClientTLSTest
@Override
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected SSLEngineResult unwrap(SSLEngine sslEngine, ByteBuffer input, ByteBuffer output) throws SSLException
@ -1199,9 +1198,9 @@ public class HttpClientTLSTest
return new SslClientConnectionFactory(sslContextFactory, getByteBufferPool(), getExecutor(), connectionFactory)
{
@Override
protected SslConnection newSslConnection(ByteBufferPool bufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(bufferPool, executor, endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected SSLEngineResult wrap(SSLEngine sslEngine, ByteBuffer[] input, ByteBuffer output) throws SSLException

View File

@ -49,7 +49,7 @@ public class SslConnectionTest
ByteArrayEndPoint endPoint = new ByteArrayEndPoint();
SSLEngine sslEngine = sslContextFactory.newSSLEngine();
sslEngine.setUseClientMode(false);
SslConnection sslConnection = new SslConnection(bufferPool, threadPool, endPoint, sslEngine);
SslConnection sslConnection = new SslConnection(bufferPool, threadPool, sslContextFactory, endPoint, sslEngine);
EndPoint sslEndPoint = sslConnection.getSslEndPoint();
sslEndPoint.setConnection(new AbstractConnection(sslEndPoint, threadPool)
{

View File

@ -412,17 +412,17 @@ public class HTTP2Client extends ContainerLifeCycle
connect(null, address, listener, promise);
}
public CompletableFuture<Session> connect(SslContextFactory sslContextFactory, SocketAddress address, Session.Listener listener)
public CompletableFuture<Session> connect(SslContextFactory.Client sslContextFactory, SocketAddress address, Session.Listener listener)
{
return Promise.Completable.with(p -> connect(sslContextFactory, address, listener, p));
}
public void connect(SslContextFactory sslContextFactory, SocketAddress address, Session.Listener listener, Promise<Session> promise)
public void connect(SslContextFactory.Client sslContextFactory, SocketAddress address, Session.Listener listener, Promise<Session> promise)
{
connect(sslContextFactory, address, listener, promise, null);
}
public void connect(SslContextFactory sslContextFactory, SocketAddress address, Session.Listener listener, Promise<Session> promise, Map<String, Object> context)
public void connect(SslContextFactory.Client sslContextFactory, SocketAddress address, Session.Listener listener, Promise<Session> promise, Map<String, Object> context)
{
ClientConnectionFactory factory = newClientConnectionFactory(sslContextFactory);
connect(address, factory, listener, promise, context);
@ -435,7 +435,7 @@ public class HTTP2Client extends ContainerLifeCycle
connector.connect(address, context);
}
public void accept(SslContextFactory sslContextFactory, SocketChannel channel, Session.Listener listener, Promise<Session> promise)
public void accept(SslContextFactory.Client sslContextFactory, SocketChannel channel, Session.Listener listener, Promise<Session> promise)
{
ClientConnectionFactory factory = newClientConnectionFactory(sslContextFactory);
accept(channel, factory, listener, promise);
@ -459,7 +459,7 @@ public class HTTP2Client extends ContainerLifeCycle
return context;
}
private ClientConnectionFactory newClientConnectionFactory(SslContextFactory sslContextFactory)
private ClientConnectionFactory newClientConnectionFactory(SslContextFactory.Client sslContextFactory)
{
ClientConnectionFactory factory = new HTTP2ClientConnectionFactory();
if (sslContextFactory != null)

View File

@ -43,7 +43,6 @@ import org.eclipse.jetty.http2.parser.ServerParser;
import org.eclipse.jetty.http2.parser.SettingsBodyParser;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.ssl.SslConnection;
import org.eclipse.jetty.server.ConnectionMetaData;
import org.eclipse.jetty.server.Connector;
import org.eclipse.jetty.server.HttpChannel;
@ -392,12 +391,6 @@ public class HTTP2ServerConnection extends HTTP2Connection implements Connection
return true;
}
@Override
public boolean isSecure()
{
return getEndPoint() instanceof SslConnection.SslEndPoint;
}
@Override
public boolean isPushSupported()
{

View File

@ -36,6 +36,7 @@ import org.eclipse.jetty.server.HttpConfiguration;
import org.eclipse.jetty.server.ProxyConnectionFactory;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.Response;
import org.eclipse.jetty.server.SecureRequestCustomizer;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.util.Callback;
@ -46,7 +47,10 @@ import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.sameInstance;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ -61,6 +65,7 @@ public class ProxyProtocolTest
{
server = new Server();
HttpConfiguration configuration = new HttpConfiguration();
configuration.addCustomizer(new SecureRequestCustomizer());
connector = new ServerConnector(server, new ProxyConnectionFactory(), new HTTP2CServerConnectionFactory(configuration));
server.addConnector(connector);
server.setHandler(handler);
@ -130,6 +135,8 @@ public class ProxyProtocolTest
@Override
public boolean handle(Request request, Response response, Callback callback)
{
assertTrue(request.isSecure());
assertTrue(request.getConnectionMetaData().isSecure());
assertEquals("10.0.0.4", Request.getRemoteAddr(request));
assertEquals(33824, Request.getRemotePort(request));
assertEquals("10.0.0.5", Request.getLocalAddr(request));
@ -137,7 +144,7 @@ public class ProxyProtocolTest
EndPoint endPoint = request.getConnectionMetaData().getConnection().getEndPoint();
assertThat(endPoint, instanceOf(ProxyConnectionFactory.ProxyEndPoint.class));
ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = (ProxyConnectionFactory.ProxyEndPoint)endPoint;
assertNotNull(proxyEndPoint.getAttribute(ProxyConnectionFactory.TLS_VERSION));
assertNotNull(proxyEndPoint.getSslSessionData());
callback.succeeded();
return true;
}
@ -171,4 +178,80 @@ public class ProxyProtocolTest
});
assertTrue(latch.await(5, TimeUnit.SECONDS));
}
@Test
public void testProxyGetV2Cipher() throws Exception
{
startServer(new Handler.Abstract()
{
@Override
public boolean handle(Request request, Response response, Callback callback)
{
assertTrue(request.isSecure());
assertTrue(request.getConnectionMetaData().isSecure());
assertEquals("10.0.0.4", Request.getRemoteAddr(request));
assertEquals(33824, Request.getRemotePort(request));
assertEquals("10.0.0.5", Request.getLocalAddr(request));
assertEquals(8888, Request.getLocalPort(request));
EndPoint endPoint = request.getConnectionMetaData().getConnection().getEndPoint();
assertThat(endPoint, instanceOf(ProxyConnectionFactory.ProxyEndPoint.class));
ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = (ProxyConnectionFactory.ProxyEndPoint)endPoint;
EndPoint.SslSessionData sslSessionData = proxyEndPoint.getSslSessionData();
assertThat(sslSessionData, notNullValue());
assertThat(sslSessionData.cipherSuite(), equalTo("TEST_128_XYZ"));
assertThat(request.getAttribute(EndPoint.SslSessionData.ATTRIBUTE), sameInstance(sslSessionData));
callback.succeeded();
return true;
}
});
String request1 =
"0D0A0D0A000D0A515549540A" + // MAGIC
"21" + // Version | Command = PROXY
"11" + // FAM = AF_INET | PROT = STREAM
"0032" + // length = 4+4+2+2+1+2+26+9
"0A000004" + // SRC_ADDR 10.0.0.4
"0A000005" + // DST_ADDR 10.0.0.5
"8420" + // SRC_PORT 33824
"22B8" + // DST_PORT 8888
"20" + // Type PP2_TYPE_SSL
"001A" + // length 26 = 1+4+1+2+3+1+2+12
"01" + // client PP2_CLIENT_SSL
"00000000" + // verify 0 == verified
"21" + // PP2_SUBTYPE_SSL_VERSION
"0003" + // len = 3
"312E32" + // version string "1.2"
"23" + // type PP2_SUBTYPE_SSL_CIPHER
"000C" + // length = 12
"544553545F3132385F58595A" + // cipher "TEST_128_XYZ"
"05" + // type PP2_TYPE_UNIQUE_ID
"0006" + // length = 6
"466f6f426172" // value "FooBar" in hex
;
SocketChannel channel = SocketChannel.open();
channel.connect(new InetSocketAddress("localhost", connector.getLocalPort()));
channel.write(ByteBuffer.wrap(StringUtil.fromHexString(request1)));
FuturePromise<Session> promise = new FuturePromise<>();
client.accept(null, channel, new Session.Listener() {}, promise);
Session session = promise.get(5, TimeUnit.SECONDS);
String uri = "http://localhost:" + connector.getLocalPort() + "/";
MetaData.Request metaData = new MetaData.Request("GET", HttpURI.from(uri), HttpVersion.HTTP_2, HttpFields.EMPTY);
HeadersFrame frame = new HeadersFrame(metaData, null, true);
CountDownLatch latch = new CountDownLatch(1);
session.newStream(frame, new Promise.Adapter<>(), new Stream.Listener()
{
@Override
public void onHeaders(Stream stream, HeadersFrame frame)
{
MetaData.Response response = (MetaData.Response)frame.getMetaData();
assertEquals(HttpStatus.OK_200, response.getStatus());
if (frame.isEndStream())
latch.countDown();
}
});
assertTrue(latch.await(5, TimeUnit.SECONDS));
}
}

View File

@ -20,10 +20,13 @@ import java.net.SocketAddress;
import java.nio.ByteBuffer;
import java.nio.channels.ReadPendingException;
import java.nio.channels.WritePendingException;
import java.security.cert.X509Certificate;
import javax.net.ssl.SSLSession;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.FutureCallback;
import org.eclipse.jetty.util.IteratingCallback;
import org.eclipse.jetty.util.ssl.SslContextFactory;
import org.eclipse.jetty.util.thread.Invocable;
/**
@ -92,7 +95,7 @@ public interface EndPoint extends Closeable
/**
* Marks an {@code EndPoint} that wraps another {@code EndPoint}.
*/
public interface Wrapper
interface Wrapper
{
/**
* @return The wrapped {@code EndPoint}
@ -324,5 +327,116 @@ public interface EndPoint extends Closeable
*
* @param newConnection the connection to upgrade to
*/
public void upgrade(Connection newConnection);
void upgrade(Connection newConnection);
/**
* Get the SslSessionData of a secure end point.
* @return A {@link SslSessionData} instance (with possibly null field values) if secure, else {@code null}.
*/
default SslSessionData getSslSessionData()
{
return null;
}
/**
* @return whether this EndPoint represents a secure communication.
*/
default boolean isSecure()
{
return getSslSessionData() != null;
}
/**
* Interface representing bundle of SSLSession associated data.
*/
interface SslSessionData
{
/**
* The name at which an {@code SslSessionData} instance may be found as a request
* {@link org.eclipse.jetty.util.Attributes Attribute} or from {@link SSLSession#getValue(String)}.
*/
String ATTRIBUTE = "org.eclipse.jetty.io.Endpoint.SslSessionData";
/**
* @return The {@link SSLSession} itself, if known, else {@code null}.
*/
SSLSession sslSession();
/**
* @return The {@link SSLSession#getId()} rendered as a hex string, if known, else {@code null}.
*/
String sslSessionId();
/**
* @return The {@link SSLSession#getCipherSuite()} if known, else {@code null}.
*/
String cipherSuite();
/**
* @return The {@link SSLSession#getPeerCertificates()}s converted to {@link X509Certificate}, if known, else {@code null}.
*/
X509Certificate[] peerCertificates();
/**
* Calculates the key size based on the cipher suite.
* @return the key size.
*/
default int keySize()
{
String cipherSuite = cipherSuite();
return cipherSuite == null ? 0 : SslContextFactory.deduceKeyLength(cipherSuite);
}
static SslSessionData from(SSLSession sslSession, String sslSessionId, String cipherSuite, X509Certificate[] peerCertificates)
{
return new SslSessionData()
{
@Override
public SSLSession sslSession()
{
return sslSession;
}
@Override
public String sslSessionId()
{
return sslSessionId;
}
@Override
public String cipherSuite()
{
return cipherSuite;
}
@Override
public X509Certificate[] peerCertificates()
{
return peerCertificates;
}
};
}
static SslSessionData withCipherSuite(SslSessionData baseData, String cipherSuite)
{
return (baseData == null)
? from(null, null, cipherSuite, null)
: from(
baseData.sslSession(),
baseData.sslSessionId(),
cipherSuite != null ? cipherSuite : baseData.cipherSuite(),
baseData.peerCertificates());
}
static SslSessionData withSslSessionId(SslSessionData baseData, String sslSessionId)
{
return (baseData == null)
? from(null, sslSessionId, null, null)
: from(
baseData.sslSession(),
sslSessionId != null ? sslSessionId : baseData.sslSessionId(),
baseData.cipherSuite(),
baseData.peerCertificates());
}
}
}

View File

@ -39,25 +39,40 @@ public class SslClientConnectionFactory implements ClientConnectionFactory
{
public static final String SSL_ENGINE_CONTEXT_KEY = "org.eclipse.jetty.client.ssl.engine";
private final SslContextFactory sslContextFactory;
private final ByteBufferPool byteBufferPool;
private final Executor executor;
private final ClientConnectionFactory connectionFactory;
private final SslContextFactory.Client _sslContextFactory;
private final ByteBufferPool _byteBufferPool;
private final Executor _executor;
private final ClientConnectionFactory _clientConnectionFactory;
private boolean _directBuffersForEncryption = true;
private boolean _directBuffersForDecryption = true;
private boolean _requireCloseMessage;
public SslClientConnectionFactory(SslContextFactory sslContextFactory, ByteBufferPool byteBufferPool, Executor executor, ClientConnectionFactory connectionFactory)
public SslClientConnectionFactory(SslContextFactory.Client sslContextFactory, ByteBufferPool byteBufferPool, Executor executor, ClientConnectionFactory connectionFactory)
{
this.sslContextFactory = Objects.requireNonNull(sslContextFactory, "Missing SslContextFactory");
this.byteBufferPool = byteBufferPool;
this.executor = executor;
this.connectionFactory = connectionFactory;
_sslContextFactory = Objects.requireNonNull(sslContextFactory, "Missing SslContextFactory");
_byteBufferPool = byteBufferPool;
_executor = executor;
_clientConnectionFactory = connectionFactory;
}
public SslContextFactory.Client getSslContextFactory()
{
return _sslContextFactory;
}
public ByteBufferPool getByteBufferPool()
{
return _byteBufferPool;
}
public Executor getExecutor()
{
return _executor;
}
public ClientConnectionFactory getClientConnectionFactory()
{
return connectionFactory;
return _clientConnectionFactory;
}
public void setDirectBuffersForEncryption(boolean useDirectBuffers)
@ -108,21 +123,21 @@ public class SslClientConnectionFactory implements ClientConnectionFactory
InetSocketAddress inetRemote = (InetSocketAddress)remote;
String host = inetRemote.getHostString();
int port = inetRemote.getPort();
engine = sslContextFactory instanceof SslEngineFactory
? ((SslEngineFactory)sslContextFactory).newSslEngine(host, port, context)
: sslContextFactory.newSSLEngine(host, port);
engine = _sslContextFactory instanceof SslEngineFactory
? ((SslEngineFactory)_sslContextFactory).newSslEngine(host, port, context)
: _sslContextFactory.newSSLEngine(host, port);
}
else
{
engine = sslContextFactory.newSSLEngine();
engine = _sslContextFactory.newSSLEngine();
}
engine.setUseClientMode(true);
context.put(SSL_ENGINE_CONTEXT_KEY, engine);
SslConnection sslConnection = newSslConnection(byteBufferPool, executor, endPoint, engine);
SslConnection sslConnection = newSslConnection(endPoint, engine);
EndPoint appEndPoint = sslConnection.getSslEndPoint();
appEndPoint.setConnection(connectionFactory.newConnection(appEndPoint, context));
appEndPoint.setConnection(_clientConnectionFactory.newConnection(appEndPoint, context));
sslConnection.addHandshakeListener(new HTTPSHandshakeListener(context));
customize(sslConnection, context);
@ -130,9 +145,9 @@ public class SslClientConnectionFactory implements ClientConnectionFactory
return sslConnection;
}
protected SslConnection newSslConnection(ByteBufferPool byteBufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(byteBufferPool, executor, endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption());
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption());
}
@Override
@ -141,8 +156,8 @@ public class SslClientConnectionFactory implements ClientConnectionFactory
if (connection instanceof SslConnection)
{
SslConnection sslConnection = (SslConnection)connection;
sslConnection.setRenegotiationAllowed(sslContextFactory.isRenegotiationAllowed());
sslConnection.setRenegotiationLimit(sslContextFactory.getRenegotiationLimit());
sslConnection.setRenegotiationAllowed(_sslContextFactory.isRenegotiationAllowed());
sslConnection.setRenegotiationLimit(_sslContextFactory.getRenegotiationLimit());
sslConnection.setRequireCloseMessage(isRequireCloseMessage());
ContainerLifeCycle client = (ContainerLifeCycle)context.get(ClientConnectionFactory.CLIENT_CONTEXT_KEY);
if (client != null)
@ -182,7 +197,7 @@ public class SslClientConnectionFactory implements ClientConnectionFactory
@Override
public void handshakeSucceeded(Event event) throws SSLException
{
HostnameVerifier verifier = sslContextFactory.getHostnameVerifier();
HostnameVerifier verifier = _sslContextFactory.getHostnameVerifier();
if (verifier != null)
{
SocketAddress address = (SocketAddress)context.get(ClientConnector.REMOTE_SOCKET_ADDRESS_CONTEXT_KEY);

View File

@ -16,6 +16,7 @@ package org.eclipse.jetty.io.ssl;
import java.io.IOException;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Executor;
@ -43,6 +44,7 @@ import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.ExceptionUtil;
import org.eclipse.jetty.util.IO;
import org.eclipse.jetty.util.StringUtil;
import org.eclipse.jetty.util.ssl.SslContextFactory;
import org.eclipse.jetty.util.thread.AutoLock;
import org.eclipse.jetty.util.thread.Invocable;
import org.slf4j.Logger;
@ -112,6 +114,7 @@ public class SslConnection extends AbstractConnection implements Connection.Upgr
private final AtomicLong _bytesOut = new AtomicLong();
private final ByteBufferPool _bufferPool;
private final SSLEngine _sslEngine;
private final SslContextFactory _sslContextFactory;
private final SslEndPoint _sslEndPoint;
private final boolean _encryptedDirectBuffers;
private final boolean _decryptedDirectBuffers;
@ -166,22 +169,35 @@ public class SslConnection extends AbstractConnection implements Connection.Upgr
}
};
@Deprecated
public SslConnection(ByteBufferPool byteBufferPool, Executor executor, EndPoint endPoint, SSLEngine sslEngine)
{
this(byteBufferPool, executor, endPoint, sslEngine, false, false);
this(byteBufferPool, executor, null, endPoint, sslEngine, false, false);
}
public SslConnection(ByteBufferPool byteBufferPool, Executor executor, EndPoint endPoint, SSLEngine sslEngine,
public SslConnection(ByteBufferPool byteBufferPool, Executor executor, SslContextFactory sslContextFactory, EndPoint endPoint, SSLEngine sslEngine)
{
this(byteBufferPool, executor, sslContextFactory, endPoint, sslEngine, false, false);
}
@Deprecated
public SslConnection(ByteBufferPool byteBufferPool, Executor executor, EndPoint endPoint, SSLEngine sslEngine, boolean useDirectBuffersForEncryption, boolean useDirectBuffersForDecryption)
{
this(byteBufferPool, executor, null, endPoint, sslEngine, useDirectBuffersForEncryption, useDirectBuffersForDecryption);
}
public SslConnection(ByteBufferPool byteBufferPool, Executor executor, SslContextFactory sslContextFactory, EndPoint endPoint, SSLEngine sslEngine,
boolean useDirectBuffersForEncryption, boolean useDirectBuffersForDecryption)
{
// This connection does not execute calls to onFillable(), so they will be called by the selector thread.
// onFillable() does not block and will only wakeup another thread to do the actual reading and handling.
super(endPoint, executor);
this._bufferPool = byteBufferPool;
this._sslEngine = sslEngine;
this._sslEndPoint = newSslEndPoint();
this._encryptedDirectBuffers = useDirectBuffersForEncryption;
this._decryptedDirectBuffers = useDirectBuffersForDecryption;
_bufferPool = byteBufferPool;
_sslEngine = sslEngine;
_sslEndPoint = newSslEndPoint();
_sslContextFactory = sslContextFactory;
_encryptedDirectBuffers = useDirectBuffersForEncryption;
_decryptedDirectBuffers = useDirectBuffersForDecryption;
}
@Override
@ -485,6 +501,9 @@ public class SslConnection extends AbstractConnection implements Connection.Upgr
public class SslEndPoint extends AbstractEndPoint implements EndPoint.Wrapper
{
// This is not a simple EndPoint.Wrapper because it has another set of the machinery
// from AbstractEndPoint for fillInterest and write flushing, separate to the wrapped EndPoint
private final Callback _incompleteWriteCallback = new IncompleteWriteCallback();
private Throwable _failure;
@ -1624,6 +1643,28 @@ public class SslConnection extends AbstractConnection implements Connection.Upgr
return String.format("SSL@%h.DEP.writeCallback", SslConnection.this);
}
}
@Override
public SslSessionData getSslSessionData()
{
SSLSession sslSession = _sslEngine.getSession();
SslSessionData sslSessionData = (SslSessionData)sslSession.getValue(SslSessionData.ATTRIBUTE);
if (sslSessionData == null)
{
String cipherSuite = sslSession.getCipherSuite();
X509Certificate[] peerCertificates = _sslContextFactory != null
? _sslContextFactory.getX509CertChain(sslSession)
: SslContextFactory.getCertChain(sslSession);
byte[] bytes = sslSession.getId();
String idStr = StringUtil.toHexString(bytes);
sslSessionData = SslSessionData.from(sslSession, idStr, cipherSuite, peerCertificates);
sslSession.putValue(SslSessionData.ATTRIBUTE, sslSessionData);
}
return sslSessionData;
}
}
private abstract class RunnableTask implements Invocable.Task

View File

@ -640,7 +640,7 @@ public class SocketChannelEndPointTest
{
SSLEngine engine = _sslCtxFactory.newSSLEngine();
engine.setUseClientMode(false);
SslConnection sslConnection = new SslConnection(_bufferPool, executor, endpoint, engine);
SslConnection sslConnection = new SslConnection(_bufferPool, executor, _sslCtxFactory, endpoint, engine);
sslConnection.setRenegotiationAllowed(_sslCtxFactory.isRenegotiationAllowed());
sslConnection.setRenegotiationLimit(_sslCtxFactory.getRenegotiationLimit());
SslConnection.SslEndPoint sslEndPoint = sslConnection.getSslEndPoint();

View File

@ -87,7 +87,7 @@ public class SslConnectionTest
{
SSLEngine engine = _sslCtxFactory.newSSLEngine();
engine.setUseClientMode(false);
SslConnection sslConnection = new SslConnection(_bufferPool, getExecutor(), endpoint, engine);
SslConnection sslConnection = new SslConnection(_bufferPool, getExecutor(), _sslCtxFactory, endpoint, engine);
sslConnection.setRenegotiationAllowed(_sslCtxFactory.isRenegotiationAllowed());
sslConnection.setRenegotiationLimit(_sslCtxFactory.getRenegotiationLimit());
SslConnection.SslEndPoint sslEndPoint = sslConnection.getSslEndPoint();

View File

@ -19,14 +19,13 @@ import java.util.Base64;
import java.util.Objects;
import org.eclipse.jetty.http.HttpStatus;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.security.AuthenticationState;
import org.eclipse.jetty.security.Authenticator;
import org.eclipse.jetty.security.ServerAuthException;
import org.eclipse.jetty.security.UserIdentity;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.Response;
import org.eclipse.jetty.server.SecureRequestCustomizer;
import org.eclipse.jetty.server.SecureRequestCustomizer.SslSessionData;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.ssl.SslContextFactory;
@ -56,8 +55,7 @@ public class SslClientCertAuthenticator extends LoginAuthenticator
@Override
public AuthenticationState validateRequest(Request req, Response res, Callback callback) throws ServerAuthException
{
SslSessionData sslSessionData = (SslSessionData)req.getAttribute(SecureRequestCustomizer.DEFAULT_SSL_SESSION_DATA_ATTRIBUTE);
if (sslSessionData == null)
if (!(req.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData))
{
Response.writeError(req, res, callback, HttpStatus.FORBIDDEN_403);
return AuthenticationState.SEND_FAILURE;

View File

@ -18,6 +18,7 @@ import java.net.SocketAddress;
import org.eclipse.jetty.http.HttpVersion;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.util.Attributes;
import org.eclipse.jetty.util.HostPort;
@ -43,7 +44,11 @@ public interface ConnectionMetaData extends Attributes
boolean isPersistent();
boolean isSecure();
default boolean isSecure()
{
EndPoint endPoint = getConnection().getEndPoint();
return endPoint != null && endPoint.isSecure();
}
/**
* @return whether the functionality of pushing resources is supported

View File

@ -18,6 +18,7 @@ import java.lang.invoke.MethodHandles;
import java.lang.invoke.MethodType;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.util.Set;
import org.eclipse.jetty.http.BadMessageException;
import org.eclipse.jetty.http.HostPortHttpField;
@ -27,7 +28,9 @@ import org.eclipse.jetty.http.HttpHeader;
import org.eclipse.jetty.http.HttpScheme;
import org.eclipse.jetty.http.HttpURI;
import org.eclipse.jetty.http.QuotedCSVParser;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.server.internal.HttpConnection;
import org.eclipse.jetty.util.Attributes;
import org.eclipse.jetty.util.HostPort;
import org.eclipse.jetty.util.Index;
import org.eclipse.jetty.util.StringUtil;
@ -609,7 +612,23 @@ public class ForwardedRequestCustomizer implements HttpConfiguration.Customizer
? request.getHeaders()
: HttpFields.build(request.getHeaders(), authority);
return new Request.Wrapper(request)
EndPoint.SslSessionData sslSessionData = forwarded._sslSessionData;
return new Request.AttributesWrapper(request, sslSessionData == null ? request : new Attributes.Synthetic(request)
{
private static final Set<String> ATTRIBUTES = Set.of(EndPoint.SslSessionData.ATTRIBUTE);
@Override
protected Object getSyntheticAttribute(String name)
{
return EndPoint.SslSessionData.ATTRIBUTE.equals(name) ? sslSessionData : null;
}
@Override
protected Set<String> getSyntheticNameSet()
{
return ATTRIBUTES;
}
})
{
@Override
public HttpURI getHttpURI()
@ -824,6 +843,7 @@ public class ForwardedRequestCustomizer implements HttpConfiguration.Customizer
Source _protoSource = Source.UNSET;
Boolean _secure;
boolean _secureScheme = false;
EndPoint.SslSessionData _sslSessionData;
public Forwarded(Request request, HttpConfiguration config)
{
@ -872,7 +892,7 @@ public class ForwardedRequestCustomizer implements HttpConfiguration.Customizer
*/
public void handleCipherSuite(HttpField field)
{
_request.setAttribute("jakarta.servlet.request.cipher_suite", field.getValue());
_sslSessionData = EndPoint.SslSessionData.withCipherSuite(_sslSessionData, field.getValue());
// Is ForwardingRequestCustomizer configured to trigger isSecure and scheme change on this header?
if (isSslIsSecure())
@ -888,7 +908,7 @@ public class ForwardedRequestCustomizer implements HttpConfiguration.Customizer
*/
public void handleSslSessionId(HttpField field)
{
_request.setAttribute("jakarta.servlet.request.ssl_session_id", field.getValue());
_sslSessionData = EndPoint.SslSessionData.withSslSessionId(_sslSessionData, field.getValue());
// Is ForwardingRequestCustomizer configured to trigger isSecure and scheme change on this header?
if (isSslIsSecure())

View File

@ -182,6 +182,11 @@ public class HttpConfiguration implements Dumpable
return null;
}
public boolean removeCustomizer(Customizer customizer)
{
return _customizers.remove(customizer);
}
@ManagedAttribute("The size in bytes of the output buffer used to aggregate HTTP output")
public int getOutputBufferSize()
{

View File

@ -30,7 +30,6 @@ import org.eclipse.jetty.io.AbstractConnection;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.RetainableByteBuffer;
import org.eclipse.jetty.util.Attributes;
import org.eclipse.jetty.util.BufferUtil;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.StringUtil;
@ -46,7 +45,6 @@ import org.slf4j.LoggerFactory;
*/
public class ProxyConnectionFactory extends DetectorConnectionFactory
{
public static final String TLS_VERSION = "TLS_VERSION";
private static final Logger LOG = LoggerFactory.getLogger(ProxyConnectionFactory.class);
public ProxyConnectionFactory()
@ -380,6 +378,23 @@ public class ProxyConnectionFactory extends DetectorConnectionFactory
0x0D, 0x0A, 0x0D, 0x0A, 0x00, 0x0D, 0x0A, 0x51, 0x55, 0x49, 0x54, 0x0A
};
private static final int PP2_TYPE_ALPN = 0x01;
private static final int PP2_TYPE_AUTHORITY = 0x02;
private static final int PP2_TYPE_CRC32C = 0x03;
private static final int PP2_TYPE_NOOP = 0x04;
private static final int PP2_TYPE_UNIQUE_ID = 0x05;
private static final int PP2_TYPE_SSL = 0x20;
private static final int PP2_SUBTYPE_SSL_VERSION = 0x21;
private static final int PP2_SUBTYPE_SSL_CN = 0x22;
private static final int PP2_SUBTYPE_SSL_CIPHER = 0x23;
private static final int PP2_SUBTYPE_SSL_SIG_ALG = 0x24;
private static final int PP2_SUBTYPE_SSL_KEY_ALG = 0x25;
private static final int PP2_TYPE_NETNS = 0x30;
private static final int PP2_CLIENT_SSL = 0x01;
private static final int PP2_CLIENT_CERT_CONN = 0x02;
private static final int PP2_CLIENT_CERT_SESS = 0x04;
private final String _nextProtocol;
private int _maxProxyHeader = 1024;
@ -629,7 +644,10 @@ public class ProxyConnectionFactory extends DetectorConnectionFactory
}
default -> throw new IllegalStateException("Unsupported family " + _family);
}
proxyEndPoint = new ProxyEndPoint(endPoint, local, remote);
int client = 0;
String sslCipher = null;
Map<Integer, byte[]> tlvs = null;
// Any additional info?
while (byteBuffer.remaining() > nonProxyRemaining)
@ -643,32 +661,34 @@ public class ProxyConnectionFactory extends DetectorConnectionFactory
LOG.debug(String.format("Proxy v2 T=%x L=%d V=%s for %s", type, length, StringUtil.toHexString(value), this));
// PP2_TYPE_NOOP is only used for byte alignment, skip them.
if (type != ProxyEndPoint.PP2_TYPE_NOOP)
proxyEndPoint.putTLV(type, value);
if (type == ProxyEndPoint.PP2_TYPE_SSL)
if (type != PP2_TYPE_NOOP)
{
int client = value[0] & 0xFF;
if (client == ProxyEndPoint.PP2_TYPE_SSL_PP2_CLIENT_SSL)
if (tlvs == null)
tlvs = new HashMap<>();
tlvs.put(type, value);
}
if (type == PP2_TYPE_SSL)
{
client = value[0] & 0xFF;
sslCipher = null;
int i = 5; // Index of the first sub_tlv, after verify.
while (i < length)
{
int i = 5; // Index of the first sub_tlv, after verify.
while (i < length)
{
int subType = value[i++] & 0xFF;
int subLength = (value[i++] & 0xFF) * 256 + (value[i++] & 0xFF);
byte[] subValue = new byte[subLength];
System.arraycopy(value, i, subValue, 0, subLength);
i += subLength;
if (subType == ProxyEndPoint.PP2_SUBTYPE_SSL_VERSION)
{
String tlsVersion = new String(subValue, StandardCharsets.US_ASCII);
proxyEndPoint.setAttribute(TLS_VERSION, tlsVersion);
}
}
int subType = value[i++] & 0xFF;
int subLength = (value[i++] & 0xFF) * 256 + (value[i++] & 0xFF);
byte[] subValue = new byte[subLength];
System.arraycopy(value, i, subValue, 0, subLength);
tlvs.put(subType, subValue);
i += subLength;
if (subType == PP2_SUBTYPE_SSL_CIPHER)
sslCipher = new String(subValue, StandardCharsets.US_ASCII);
}
}
}
proxyEndPoint = new ProxyEndPoint(endPoint, local, remote, tlvs, client == 0 ? null : EndPoint.SslSessionData.from(null, null, sslCipher, null));
if (LOG.isDebugEnabled())
LOG.debug("Proxy v2 {} {}", endPoint, proxyEndPoint);
}
@ -748,29 +768,32 @@ public class ProxyConnectionFactory extends DetectorConnectionFactory
}
}
public static class ProxyEndPoint extends Attributes.Lazy implements EndPoint, EndPoint.Wrapper
public static class ProxyEndPoint implements EndPoint, EndPoint.Wrapper
{
private static final int PP2_TYPE_NOOP = 0x04;
private static final int PP2_TYPE_SSL = 0x20;
private static final int PP2_TYPE_SSL_PP2_CLIENT_SSL = 0x01;
private static final int PP2_SUBTYPE_SSL_VERSION = 0x21;
private final EndPoint _endPoint;
private final SocketAddress _local;
private final SocketAddress _remote;
private Map<Integer, byte[]> _tlvs;
@Deprecated
public ProxyEndPoint(EndPoint endPoint, InetSocketAddress remote, InetSocketAddress local)
{
this(endPoint, (SocketAddress)local, remote);
}
private final Map<Integer, byte[]> _tlvs;
private final SslSessionData _sslSessionData;
public ProxyEndPoint(EndPoint endPoint, SocketAddress local, SocketAddress remote)
{
this(endPoint, local, remote, null, null);
}
public ProxyEndPoint(EndPoint endPoint, SocketAddress local, SocketAddress remote, Map<Integer, byte[]> tlvs, SslSessionData sslSessionData)
{
_endPoint = endPoint;
_local = local;
_remote = remote;
_tlvs = tlvs;
_sslSessionData = sslSessionData;
}
@Override
public SslSessionData getSslSessionData()
{
return _sslSessionData;
}
public EndPoint unwrap()
@ -778,23 +801,10 @@ public class ProxyConnectionFactory extends DetectorConnectionFactory
return _endPoint;
}
/**
* <p>Sets a TLV vector, see section 2.2.7 of the PROXY protocol specification.</p>
*
* @param type the TLV type
* @param value the TLV value
*/
private void putTLV(int type, byte[] value)
{
if (_tlvs == null)
_tlvs = new HashMap<>();
_tlvs.put(type, value);
}
/**
* <p>Gets a TLV vector, see section 2.2.7 of the PROXY protocol specification.</p>
*
* @param type the TLV type
* @param type the TLV type or subtype
* @return the TLV value or null if not present.
*/
public byte[] getTLV(int type)

View File

@ -797,11 +797,11 @@ public interface Request extends Attributes, Content.Source
* (one in {@link Attributes.Wrapper} as type {@link Attributes} and one in {@link Request.Wrapper} as
* type {@link Request}) to save the costly cast from interface type to another.
*/
private final Request _wrapped;
private final Request _request;
public Wrapper(Request wrapped)
{
_wrapped = Objects.requireNonNull(wrapped);
_request = Objects.requireNonNull(wrapped);
}
@Override
@ -974,7 +974,7 @@ public interface Request extends Attributes, Content.Source
public Request getWrapped()
{
return _wrapped;
return _request;
}
}
@ -1065,4 +1065,59 @@ public interface Request extends Attributes, Content.Source
return null;
}
}
/**
* <p>A {@link Request.Wrapper} that separately provides the request {@link Attributes}.</p>
* <p>The provided {@link Attributes} should be an {@link Attributes.Wrapper} over the request.</p>
*/
class AttributesWrapper extends Wrapper
{
private final Attributes _attributes;
/**
* @param wrapped the request to wrap
* @param attributes the provided request attributes, typically a {@link Attributes.Wrapper} over the request
*/
public AttributesWrapper(Request wrapped, Attributes attributes)
{
super(wrapped);
_attributes = Objects.requireNonNull(attributes);
}
@Override
public Map<String, Object> asAttributeMap()
{
return _attributes.asAttributeMap();
}
@Override
public void clearAttributes()
{
_attributes.clearAttributes();
}
@Override
public Object removeAttribute(String name)
{
return _attributes.removeAttribute(name);
}
@Override
public Object setAttribute(String name, Object attribute)
{
return _attributes.setAttribute(name, attribute);
}
@Override
public Object getAttribute(String name)
{
return _attributes.getAttribute(name);
}
@Override
public Set<String> getAttributeNameSet()
{
return _attributes.getAttributeNameSet();
}
}
}

View File

@ -15,12 +15,9 @@ package org.eclipse.jetty.server;
import java.security.cert.Certificate;
import java.security.cert.X509Certificate;
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine;
import javax.net.ssl.SSLSession;
import org.eclipse.jetty.http.BadMessageException;
@ -29,9 +26,6 @@ import org.eclipse.jetty.http.HttpFields;
import org.eclipse.jetty.http.HttpHeader;
import org.eclipse.jetty.http.PreEncodedHttpField;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.ssl.SslConnection;
import org.eclipse.jetty.io.ssl.SslConnection.SslEndPoint;
import org.eclipse.jetty.util.StringUtil;
import org.eclipse.jetty.util.annotation.Name;
import org.eclipse.jetty.util.ssl.SslContextFactory;
import org.eclipse.jetty.util.ssl.X509;
@ -39,53 +33,21 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* <p>Customizer that extracts the attribute from an {@link SSLContext}
* and sets them on the request with {@link Request#setAttribute(String, Object)}
* according to Servlet Specification Requirements.</p>
* <p>Customizer that extracts the attribute of an {@link SSLContext}
* and makes them available via {@link Request#getAttribute(String)}
* using the names:
* <ul>
* <li>{@link EndPoint.SslSessionData#ATTRIBUTE} for {@link EndPoint.SslSessionData}</li>
* <li>{@link #X509_ATTRIBUTE} for the local certificate as a {@link X509} instance</li>
* </ul>
* @see EndPoint.SslSessionData
*/
public class SecureRequestCustomizer implements HttpConfiguration.Customizer
{
/**
* <p>The request attribute name to use to obtain the cipher suite name.</p>
*/
public static final String CIPHER_SUITE_ATTRIBUTE = "org.eclipse.jetty.server.cipher";
/**
* <p>The request attribute name to use to obtain the key size.</p>
*/
public static final String KEY_SIZE_ATTRIBUTE = "org.eclipse.jetty.server.keySize";
/**
* <p>The request attribute name to use to obtain the {@link SSLSession#getId()}.</p>
*/
public static final String SSL_SESSION_ID_ATTRIBUTE = "org.eclipse.jetty.server.sslSessionId";
/**
* <p>The request attribute name to use to obtain the peer certificate
* chain as an array of {@link X509Certificate} objects.</p>
*/
public static final String PEER_CERTIFICATES_ATTRIBUTE = "org.eclipse.jetty.server.peerCertificates";
/**
* <p>The request attribute name to use to obtain the local certificate
* as an {@link X509} object.</p>
*/
public static final String X509_ATTRIBUTE = "org.eclipse.jetty.server.x509";
/**
* <p>The default value of the request attribute name to use to obtain
* the {@link SSLSession} object.</p>
*
* @see #setSslSessionAttribute(String)
*/
public static final String DEFAULT_SSL_SESSION_ATTRIBUTE = "org.eclipse.jetty.server.sslSession";
/**
* <p>The default value of the request attribute name to use to obtain
* the {@link SslSessionData} object.</p>
*
* @see #getSslSessionDataAttribute()
*/
public static final String DEFAULT_SSL_SESSION_DATA_ATTRIBUTE = newSslSessionDataAttribute(DEFAULT_SSL_SESSION_ATTRIBUTE);
private static final Logger LOG = LoggerFactory.getLogger(SecureRequestCustomizer.class);
private String _sslSessionAttribute = DEFAULT_SSL_SESSION_ATTRIBUTE;
private String _sslSessionDataAttribute = DEFAULT_SSL_SESSION_DATA_ATTRIBUTE;
private boolean _sniRequired;
private boolean _sniHostCheck;
private long _stsMaxAge;
@ -230,17 +192,9 @@ public class SecureRequestCustomizer implements HttpConfiguration.Customizer
public Request customize(Request request, HttpFields.Mutable responseHeaders)
{
EndPoint endPoint = request.getConnectionMetaData().getConnection().getEndPoint();
if (endPoint instanceof SslEndPoint sslEndPoint)
{
SslConnection sslConnection = sslEndPoint.getSslConnection();
SSLEngine sslEngine = sslConnection.getSSLEngine();
request = newSecureRequest(request, sslEngine);
}
else if (endPoint instanceof ProxyConnectionFactory.ProxyEndPoint proxyEndPoint)
{
if (proxyEndPoint.getAttribute(ProxyConnectionFactory.TLS_VERSION) != null)
request = newSecureRequest(request, null);
}
EndPoint.SslSessionData sslSessionData = endPoint != null ? endPoint.getSslSessionData() : null;
if (sslSessionData != null)
request = newSecureRequest(request, sslSessionData);
if (_stsField != null)
responseHeaders.add(_stsField);
@ -248,52 +202,11 @@ public class SecureRequestCustomizer implements HttpConfiguration.Customizer
return request;
}
protected Request newSecureRequest(Request request, SSLEngine sslEngine)
protected Request newSecureRequest(Request request, EndPoint.SslSessionData sslSessionData)
{
if (sslEngine != null)
return new SecureRequestWithTLSData(request, sslEngine);
else
return new SecureRequest(request);
}
private X509Certificate[] getCertChain(Connector connector, SSLSession sslSession)
{
// The in-use SslContextFactory should be present in the Connector's SslConnectionFactory
SslConnectionFactory sslConnectionFactory = connector.getConnectionFactory(SslConnectionFactory.class);
if (sslConnectionFactory != null)
{
SslContextFactory sslContextFactory = sslConnectionFactory.getSslContextFactory();
if (sslContextFactory != null)
return sslContextFactory.getX509CertChain(sslSession);
}
// Fallback, either no SslConnectionFactory or no SslContextFactory instance found
return SslContextFactory.getCertChain(sslSession);
}
public void setSslSessionAttribute(String attribute)
{
Objects.requireNonNull(attribute);
_sslSessionAttribute = attribute;
_sslSessionDataAttribute = newSslSessionDataAttribute(attribute);
}
public String getSslSessionAttribute()
{
return _sslSessionAttribute;
}
/**
* @return {@link #getSslSessionAttribute()} {@code + "Data"}
*/
public String getSslSessionDataAttribute()
{
return _sslSessionDataAttribute;
}
private static String newSslSessionDataAttribute(String sslSessionAttribute)
{
return sslSessionAttribute + "Data";
if (sslSessionData.sslSession() != null)
checkSni(request, sslSessionData.sslSession());
return new SecureRequestWithSslSessionData(request, sslSessionData);
}
protected void checkSni(Request request, SSLSession session)
@ -351,92 +264,40 @@ public class SecureRequestCustomizer implements HttpConfiguration.Customizer
}
}
protected class SecureRequestWithTLSData extends SecureRequest
protected class SecureRequestWithSslSessionData extends Request.AttributesWrapper
{
private final SSLSession _sslSession;
private final SslSessionData _sslSessionData;
private static final Set<String> ATTRIBUTES = Set.of(
EndPoint.SslSessionData.ATTRIBUTE,
X509_ATTRIBUTE
);
public SecureRequestWithTLSData(Request request, SSLEngine sslEngine)
protected SecureRequestWithSslSessionData(Request request, EndPoint.SslSessionData sslSessionData)
{
super(request);
_sslSession = sslEngine.getSession();
checkSni(request, _sslSession);
String key = SslSessionData.class.getName();
SslSessionData sslSessionData = (SslSessionData)_sslSession.getValue(key);
if (sslSessionData == null)
super(request, new Synthetic(request)
{
try
@Override
protected Object getSyntheticAttribute(String name)
{
String cipherSuite = _sslSession.getCipherSuite();
int keySize = SslContextFactory.deduceKeyLength(cipherSuite);
X509Certificate[] peerCertificates = getCertChain(getConnectionMetaData().getConnector(), _sslSession);
byte[] bytes = _sslSession.getId();
String idStr = StringUtil.toHexString(bytes);
sslSessionData = new SslSessionData(idStr, cipherSuite, keySize, peerCertificates);
_sslSession.putValue(key, sslSessionData);
return switch (name)
{
case EndPoint.SslSessionData.ATTRIBUTE -> sslSessionData;
case X509_ATTRIBUTE -> getX509(sslSessionData.sslSession());
default -> null;
};
}
catch (Exception e)
@Override
protected Set<String> getSyntheticNameSet()
{
LOG.warn("Unable to get secure details ", e);
return ATTRIBUTES;
}
}
_sslSessionData = sslSessionData;
});
}
@Override
public Object getAttribute(String name)
public boolean isSecure()
{
String sessionAttribute = getSslSessionAttribute();
if (StringUtil.isNotBlank(sessionAttribute) && name.startsWith(sessionAttribute))
{
if (name.equals(sessionAttribute))
return _sslSession;
if (name.equals(getSslSessionDataAttribute()))
return _sslSessionData;
}
return switch (name)
{
case CIPHER_SUITE_ATTRIBUTE -> _sslSessionData != null ? _sslSessionData.cipherSuite() : null;
case KEY_SIZE_ATTRIBUTE -> _sslSessionData != null ? _sslSessionData.keySize() : null;
case SSL_SESSION_ID_ATTRIBUTE -> _sslSessionData != null ? _sslSessionData.sessionId() : null;
case PEER_CERTIFICATES_ATTRIBUTE -> _sslSessionData != null ? _sslSessionData.peerCertificates() : null;
case X509_ATTRIBUTE -> getX509(_sslSession);
default -> super.getAttribute(name);
};
return true;
}
@Override
public Set<String> getAttributeNameSet()
{
Set<String> names = new HashSet<>(super.getAttributeNameSet());
if (getX509(_sslSession) != null)
names.add(X509_ATTRIBUTE);
String sessionAttribute = getSslSessionAttribute();
if (!StringUtil.isNotBlank(sessionAttribute))
{
names.add(sessionAttribute);
if (_sslSessionData != null)
{
names.add(getSslSessionDataAttribute());
names.add(CIPHER_SUITE_ATTRIBUTE);
names.add(KEY_SIZE_ATTRIBUTE);
names.add(SSL_SESSION_ID_ATTRIBUTE);
names.add(PEER_CERTIFICATES_ATTRIBUTE);
}
}
return names;
}
}
/**
* Simple bundle of data that is cached in the SSLSession.
*/
public record SslSessionData(String sessionId, String cipherSuite, int keySize, X509Certificate[] peerCertificates)
{
}
}

View File

@ -21,7 +21,6 @@ import javax.net.ssl.SSLSession;
import org.eclipse.jetty.http.HttpVersion;
import org.eclipse.jetty.io.AbstractConnection;
import org.eclipse.jetty.io.ByteBufferPool;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.ssl.SslConnection;
@ -166,8 +165,7 @@ public class SslConnectionFactory extends AbstractConnectionFactory implements C
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
ByteBufferPool byteBufferPool = connector.getByteBufferPool();
return new SslConnection(byteBufferPool, connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption());
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption());
}
@Override

View File

@ -24,7 +24,6 @@ import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -46,6 +45,7 @@ import org.eclipse.jetty.io.RetainableByteBuffer;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.Response;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.util.Attributes;
import org.eclipse.jetty.util.BufferUtil;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.StringUtil;
@ -504,18 +504,32 @@ public class ErrorHandler implements Request.Handler
return errorHandler;
}
public static class ErrorRequest extends Request.Wrapper
public static class ErrorRequest extends Request.AttributesWrapper
{
private final int _status;
private final String _message;
private final Throwable _cause;
private static final Set<String> ATTRIBUTES = Set.of(ERROR_MESSAGE, ERROR_EXCEPTION, ERROR_STATUS);
public ErrorRequest(Request request, int status, String message, Throwable cause)
{
super(request);
_status = status;
_message = message;
_cause = cause;
super(request, new Attributes.Synthetic(request)
{
@Override
protected Object getSyntheticAttribute(String name)
{
return switch (name)
{
case ERROR_MESSAGE -> message;
case ERROR_EXCEPTION -> cause;
case ERROR_STATUS -> status;
default -> null;
};
}
@Override
protected Set<String> getSyntheticNameSet()
{
return ATTRIBUTES;
}
});
}
@Override
@ -530,31 +544,6 @@ public class ErrorHandler implements Request.Handler
demandCallback.run();
}
@Override
public Object getAttribute(String name)
{
return switch (name)
{
case ERROR_MESSAGE -> _message;
case ERROR_EXCEPTION -> _cause;
case ERROR_STATUS -> _status;
default -> super.getAttribute(name);
};
}
@Override
public Set<String> getAttributeNameSet()
{
Set<String> names = new HashSet<>(super.getAttributeNameSet());
if (_message != null)
names.add(ERROR_MESSAGE);
if (_status > 0)
names.add(ERROR_STATUS);
if (_cause != null)
names.add(ERROR_EXCEPTION);
return names;
}
@Override
public String toString()
{

View File

@ -1,78 +0,0 @@
//
// ========================================================================
// Copyright (c) 1995 Mort Bay Consulting Pty Ltd and others.
//
// This program and the accompanying materials are made available under the
// terms of the Eclipse Public License v. 2.0 which is available at
// https://www.eclipse.org/legal/epl-2.0, or the Apache License, Version 2.0
// which is available at https://www.apache.org/licenses/LICENSE-2.0.
//
// SPDX-License-Identifier: EPL-2.0 OR Apache-2.0
// ========================================================================
//
package org.eclipse.jetty.server.handler;
import java.net.SocketAddress;
import org.eclipse.jetty.http.HttpURI;
import org.eclipse.jetty.server.ConnectionMetaData;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.Response;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.HostPort;
public class ProxiedRequestHandler extends Handler.Wrapper
{
@Override
public boolean handle(Request request, Response response, Callback callback) throws Exception
{
ConnectionMetaData proxiedFor = new ConnectionMetaData.Wrapper(request.getConnectionMetaData())
{
@Override
public boolean isSecure()
{
// TODO replace with value determined from headers
return super.isSecure();
}
@Override
public SocketAddress getRemoteSocketAddress()
{
// TODO replace with value determined from headers
return super.getRemoteSocketAddress();
}
@Override
public SocketAddress getLocalSocketAddress()
{
// TODO replace with value determined from headers
return super.getLocalSocketAddress();
}
@Override
public HostPort getServerAuthority()
{
// TODO replace with value determined from headers
return super.getServerAuthority();
}
};
return super.handle(new Request.Wrapper(request)
{
@Override
public HttpURI getHttpURI()
{
// TODO replace with any change in authority
return super.getHttpURI();
}
@Override
public ConnectionMetaData getConnectionMetaData()
{
return proxiedFor;
}
}, response, callback);
}
}

View File

@ -19,7 +19,6 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
@ -43,12 +42,9 @@ import org.eclipse.jetty.http.MultiPartFormData.Parts;
import org.eclipse.jetty.http.Trailers;
import org.eclipse.jetty.http.UriCompliance;
import org.eclipse.jetty.io.ByteBufferPool;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.server.Components;
import org.eclipse.jetty.server.ConnectionMetaData;
import org.eclipse.jetty.server.Connector;
import org.eclipse.jetty.server.Context;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.HttpChannel;
@ -203,24 +199,6 @@ public class HttpChannelState implements HttpChannel, Components
return _connectionMetaData;
}
// TODO: remove this
public Connection getConnection()
{
return _connectionMetaData.getConnection();
}
// TODO: remove this
public Connector getConnector()
{
return _connectionMetaData.getConnector();
}
// TODO: remove this
public EndPoint getEndPoint()
{
return getConnection().getEndPoint();
}
@Override
public ByteBufferPool getByteBufferPool()
{
@ -767,7 +745,7 @@ public class HttpChannelState implements HttpChannel, Components
}
}
public static class ChannelRequest implements Attributes, Request
public static class ChannelRequest extends Attributes.Lazy implements Request
{
private final long _headersNanoTime = NanoTime.now();
private final ChannelCallback _callback = new ChannelCallback(this);
@ -776,7 +754,6 @@ public class HttpChannelState implements HttpChannel, Components
private final MetaData.Request _metaData;
private final AutoLock _lock;
private final LongAdder _contentBytesRead = new LongAdder();
private final Attributes _attributes = new Attributes.Lazy();
private final AtomicReference<List<Consumer<Throwable>>> _onCompletion = new AtomicReference<>();
private HttpChannelState _httpChannelState;
private Request _loggedRequest;
@ -811,50 +788,6 @@ public class HttpChannelState implements HttpChannel, Components
return _contentBytesRead.longValue();
}
@Override
public Object getAttribute(String name)
{
if (name.startsWith("org.eclipse.jetty"))
{
if (Server.class.getName().equals(name))
return getConnectionMetaData().getConnector().getServer();
if (HttpChannelState.class.getName().equals(name))
return getHttpChannelState();
// TODO: is the instanceof needed?
// TODO: possibly remove this if statement or move to Servlet.
if (HttpConnection.class.getName().equals(name) &&
getConnectionMetaData().getConnection() instanceof HttpConnection)
return getConnectionMetaData().getConnection();
}
return _attributes.getAttribute(name);
}
@Override
public Object removeAttribute(String name)
{
return _attributes.removeAttribute(name);
}
@Override
public Object setAttribute(String name, Object attribute)
{
if (Server.class.getName().equals(name) || HttpChannelState.class.getName().equals(name) || HttpConnection.class.getName().equals(name))
return null;
return _attributes.setAttribute(name, attribute);
}
@Override
public Set<String> getAttributeNameSet()
{
return _attributes.getAttributeNameSet();
}
@Override
public void clearAttributes()
{
_attributes.clearAttributes();
}
@Override
public String getId()
{
@ -991,27 +924,27 @@ public class HttpChannelState implements HttpChannel, Components
{
boolean error;
HttpStream stream;
HttpChannelState httpChannelState;
try (AutoLock ignored = _lock.lock())
{
HttpChannelState httpChannel = lockedGetHttpChannelState();
httpChannelState = lockedGetHttpChannelState();
if (LOG.isDebugEnabled())
LOG.debug("demand {}", httpChannel);
LOG.debug("demand {}", httpChannelState);
error = httpChannel._failure != null;
error = httpChannelState._failure != null;
if (!error)
{
if (httpChannel._onContentAvailable != null)
if (httpChannelState._onContentAvailable != null)
throw new IllegalArgumentException("demand pending");
httpChannel._onContentAvailable = demandCallback;
httpChannelState._onContentAvailable = demandCallback;
}
stream = httpChannel._stream;
stream = httpChannelState._stream;
}
if (error)
// TODO: can we avoid re-grabbing the lock to get the HttpChannel?
getHttpChannelState()._serializedInvoker.run(demandCallback);
httpChannelState._serializedInvoker.run(demandCallback);
else
stream.demand();
}

View File

@ -263,12 +263,6 @@ public class HttpConnection extends AbstractConnection implements Runnable, Writ
return _generator.isPersistent(getHttpVersion());
}
@Override
public boolean isSecure()
{
return getEndPoint() instanceof SslConnection.SslEndPoint;
}
@Override
public SocketAddress getRemoteSocketAddress()
{

View File

@ -103,7 +103,7 @@ public class ExtendedServerTest extends HttpServerTestBase
public Runnable onRequest(MetaData.Request request)
{
Runnable todo = super.onRequest(request);
getRequest().setAttribute("DispatchedAt", ((ExtendedEndPoint)getEndPoint()).getLastSelected());
getRequest().setAttribute("DispatchedAt", ((ExtendedEndPoint)getConnectionMetaData().getConnection().getEndPoint()).getLastSelected());
return todo;
}
};

View File

@ -131,8 +131,11 @@ public class ForwardedRequestCustomizerTest
{
actual = new Actual();
actual.wasSecure.set(request.isSecure());
actual.sslSession.set(String.valueOf(request.getAttribute("jakarta.servlet.request.ssl_session_id")));
actual.sslCertificate.set(String.valueOf(request.getAttribute("jakarta.servlet.request.cipher_suite")));
if (request.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData)
{
actual.sslSession.set(sslSessionData.sslSessionId());
actual.sslCertificate.set(sslSessionData.cipherSuite());
}
actual.scheme.set(request.getHttpURI().getScheme());
actual.serverName.set(Request.getServerName(request));
actual.serverPort.set(Request.getServerPort(request));

View File

@ -24,7 +24,6 @@ import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.server.internal.HttpConnection;
import org.eclipse.jetty.util.Callback;
import org.eclipse.jetty.util.StringUtil;
import org.junit.jupiter.api.AfterEach;
@ -95,7 +94,7 @@ public class ProxyProtocolTest
InputStream input = socket.getInputStream();
BufferedReader reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8));
String response1 = reader.readLine();
assertTrue(response1.startsWith("HTTP/1.1 200 "));
assertThat(response1, startsWith("HTTP/1.1 200 "));
while (true)
{
if (reader.readLine().isEmpty())
@ -143,14 +142,13 @@ public class ProxyProtocolTest
return true;
}
private boolean validateEndPoint(Request request)
private boolean validateEndPoint(Request request)
{
HttpConnection con = (HttpConnection)request.getAttribute(HttpConnection.class.getName());
EndPoint endPoint = con.getEndPoint();
EndPoint endPoint = request.getConnectionMetaData().getConnection().getEndPoint();
ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = (ProxyConnectionFactory.ProxyEndPoint)endPoint;
return Arrays.equals(customE0, proxyEndPoint.getTLV(0xE0)) &&
Arrays.equals(customE1, proxyEndPoint.getTLV(0xE1)) &&
proxyEndPoint.getTLV(0xE2) == null;
Arrays.equals(customE1, proxyEndPoint.getTLV(0xE1)) &&
proxyEndPoint.getTLV(0xE2) == null;
}
});

View File

@ -95,7 +95,6 @@ public class ThreadStarvationTest
ServerConnector connector = new ServerConnector(server, null, null, pool, acceptors, selectors,
AbstractConnectionFactory.getFactories(sslContextFactory, httpConnectionFactory));
SecureRequestCustomizer secureRequestCustomer = new SecureRequestCustomizer();
secureRequestCustomer.setSslSessionAttribute("SSL_SESSION");
httpConnectionFactory.getHttpConfiguration().addCustomizer(secureRequestCustomer);
return connector;
};

View File

@ -42,7 +42,6 @@ import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.HttpConnectionFactory;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.Response;
import org.eclipse.jetty.server.SecureRequestCustomizer;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.server.SslConnectionFactory;
@ -202,7 +201,7 @@ public class SSLEngineTest
@Override
protected SslConnection newSslConnection(Connector connector, EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(connector.getByteBufferPool(), connector.getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
@Override
protected SSLEngineResult unwrap(SSLEngine sslEngine, ByteBuffer input, ByteBuffer output) throws SSLException
@ -368,10 +367,8 @@ public class SSLEngineTest
@Override
public boolean handle(Request request, Response response, Callback callback) throws Exception
{
// System.err.println("HANDLE "+request.getRequestURI());
SecureRequestCustomizer.SslSessionData sslData = (SecureRequestCustomizer.SslSessionData)
request.getAttribute(SecureRequestCustomizer.DEFAULT_SSL_SESSION_DATA_ATTRIBUTE);
String sslId = sslData.sessionId();
EndPoint.SslSessionData sslData = (EndPoint.SslSessionData)request.getAttribute(EndPoint.SslSessionData.ATTRIBUTE);
String sslId = sslData.sslSessionId();
assertNotNull(sslId);
Fields fields = Request.extractQueryParameters(request);

View File

@ -29,8 +29,10 @@ import javax.net.ssl.TrustManagerFactory;
import org.eclipse.jetty.io.ArrayByteBufferPool;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.server.AbstractConnectionFactory;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.HttpConfiguration;
import org.eclipse.jetty.server.HttpConnectionFactory;
import org.eclipse.jetty.server.HttpServerTestBase;
import org.eclipse.jetty.server.Request;
@ -48,11 +50,13 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.emptyOrNullString;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
/**
* HttpServer Tester for SSL based ServerConnector
@ -60,7 +64,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
public class ServerConnectorSslServerTest extends HttpServerTestBase
{
private SSLContext _sslContext;
private ArrayByteBufferPool.Tracking trackingBufferPool;
private ArrayByteBufferPool.Tracking _trackingBufferPool;
private HttpConfiguration _httpConfiguration;
public ServerConnectorSslServerTest()
{
@ -74,13 +79,11 @@ public class ServerConnectorSslServerTest extends HttpServerTestBase
SslContextFactory.Server sslContextFactory = new SslContextFactory.Server();
sslContextFactory.setKeyStorePath(keystorePath);
sslContextFactory.setKeyStorePassword("storepwd");
trackingBufferPool = new ArrayByteBufferPool.Tracking();
_trackingBufferPool = new ArrayByteBufferPool.Tracking();
HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory();
ServerConnector connector = new ServerConnector(_server, null, null, trackingBufferPool, 1, 1, AbstractConnectionFactory.getFactories(sslContextFactory, httpConnectionFactory));
SecureRequestCustomizer secureRequestCustomer = new SecureRequestCustomizer();
secureRequestCustomer.setSslSessionAttribute("SSL_SESSION");
httpConnectionFactory.getHttpConfiguration().addCustomizer(secureRequestCustomer);
_httpConfiguration = httpConnectionFactory.getHttpConfiguration();
ServerConnector connector = new ServerConnector(_server, null, null, _trackingBufferPool, 1, 1, AbstractConnectionFactory.getFactories(sslContextFactory, httpConnectionFactory));
initServer(connector);
@ -111,7 +114,7 @@ public class ServerConnectorSslServerTest extends HttpServerTestBase
@AfterEach
public void dispose() throws Exception
{
assertThat("Server Leaks: " + trackingBufferPool.dumpLeaks(), trackingBufferPool.getLeaks().size(), Matchers.is(0));
assertThat("Server Leaks: " + _trackingBufferPool.dumpLeaks(), _trackingBufferPool.getLeaks().size(), Matchers.is(0));
}
@Override
@ -194,7 +197,7 @@ public class ServerConnectorSslServerTest extends HttpServerTestBase
assertThat(response, containsString("Hello world"));
assertThat(response, containsString("scheme='https'"));
assertThat(response, containsString("isSecure='true'"));
assertThat(response, containsString("X509Certificate='null'"));
assertThat(response, containsString("X509Certificate='false'"));
Matcher matcher = Pattern.compile("cipher_suite='([^']*)'").matcher(response);
matcher.find();
@ -220,19 +223,24 @@ public class ServerConnectorSslServerTest extends HttpServerTestBase
public boolean handle(Request request, Response response, Callback callback) throws Exception
{
response.setStatus(200);
StringBuilder out = new StringBuilder();
SSLSession session = (SSLSession)request.getAttribute("SSL_SESSION");
assertThat(request.getAttributeNameSet(), containsInAnyOrder(
EndPoint.SslSessionData.ATTRIBUTE,
SecureRequestCustomizer.X509_ATTRIBUTE));
SecureRequestCustomizer.SslSessionData data = (SecureRequestCustomizer.SslSessionData)request.getAttribute("SSL_SESSIONData");
StringBuilder out = new StringBuilder();
EndPoint.SslSessionData data = (EndPoint.SslSessionData)request.getAttribute(EndPoint.SslSessionData.ATTRIBUTE);
assertNotNull(data);
SSLSession session = data.sslSession();
out.append("Hello world").append('\n');
out.append("scheme='").append(request.getHttpURI().getScheme()).append("'").append('\n');
out.append("isSecure='").append(request.isSecure()).append("'").append('\n');
out.append("X509Certificate='").append(data == null ? "" : data.peerCertificates()).append("'").append('\n');
out.append("X509Certificate='").append(data.peerCertificates() != null).append("'").append('\n');
out.append("cipher_suite='").append(session == null ? "" : session.getCipherSuite()).append("'").append('\n');
out.append("key_size='").append(data == null ? "" : data.keySize()).append("'").append('\n');
out.append("ssl_session_id='").append(data == null ? "" : data.sessionId()).append("'").append('\n');
out.append("key_size='").append(data.keySize()).append("'").append('\n');
out.append("ssl_session_id='").append(data.sslSessionId()).append("'").append('\n');
out.append("ssl_session='").append(session).append("'").append('\n');
Content.Sink.write(response, true, out.toString(), callback);
return true;
}

View File

@ -18,7 +18,6 @@ import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.SocketTimeoutException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
@ -35,7 +34,6 @@ import org.eclipse.jetty.client.Request;
import org.eclipse.jetty.client.Response;
import org.eclipse.jetty.client.Result;
import org.eclipse.jetty.http.HttpStatus;
import org.eclipse.jetty.io.ByteBufferPool;
import org.eclipse.jetty.io.ClientConnectionFactory;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EndPoint;
@ -241,9 +239,9 @@ public class HttpClientTimeoutTest extends AbstractTest
return new SslClientConnectionFactory(sslContextFactory, getByteBufferPool(), getExecutor(), connectionFactory)
{
@Override
protected SslConnection newSslConnection(ByteBufferPool bufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(bufferPool, executor, endPoint, engine)
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine)
{
@Override
protected boolean onReadTimeout(TimeoutException timeout)

View File

@ -16,7 +16,6 @@ package org.eclipse.jetty.test.client.transport;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@ -25,7 +24,6 @@ import javax.net.ssl.SSLEngineResult;
import javax.net.ssl.SSLException;
import org.eclipse.jetty.client.HttpClient;
import org.eclipse.jetty.io.ByteBufferPool;
import org.eclipse.jetty.io.ClientConnectionFactory;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.SelectorManager;
@ -99,9 +97,9 @@ public class TLSHandshakeFailureTest extends AbstractTest
return new SslClientConnectionFactory(sslContextFactory, getByteBufferPool(), getExecutor(), connectionFactory)
{
@Override
protected SslConnection newSslConnection(ByteBufferPool byteBufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(byteBufferPool, executor, endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
private final AtomicInteger wraps = new AtomicInteger();
@ -175,9 +173,9 @@ public class TLSHandshakeFailureTest extends AbstractTest
return new SslClientConnectionFactory(sslContextFactory, getByteBufferPool(), getExecutor(), connectionFactory)
{
@Override
protected SslConnection newSslConnection(ByteBufferPool byteBufferPool, Executor executor, EndPoint endPoint, SSLEngine engine)
protected SslConnection newSslConnection(EndPoint endPoint, SSLEngine engine)
{
return new SslConnection(byteBufferPool, executor, endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
return new SslConnection(getByteBufferPool(), getExecutor(), getSslContextFactory(), endPoint, engine, isDirectBuffersForEncryption(), isDirectBuffersForDecryption())
{
private final AtomicInteger unwraps = new AtomicInteger();

View File

@ -16,8 +16,8 @@ package org.eclipse.jetty.util;
import java.io.IOException;
import java.util.AbstractMap;
import java.util.AbstractSet;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
@ -122,6 +122,12 @@ public interface Attributes
removeAttribute(name);
}
@Override
int hashCode();
@Override
boolean equals(Object o);
/** Unwrap all {@link Wrapper}s of the attributes
* @param attributes The attributes to unwrap, which may be a {@link Wrapper}.
* @return The core attributes
@ -135,6 +141,34 @@ public interface Attributes
return attributes;
}
static int hashCode(Attributes attributes)
{
int hash = 0;
for (String name : attributes.getAttributeNameSet())
hash = hash * 31 + Objects.hash(name, attributes.getAttribute(name));
return hash;
}
static boolean equals(Attributes attributes, Object o)
{
if (o instanceof Attributes a)
{
Set<String> ours = attributes.getAttributeNameSet();
Set<String> theirs = a.getAttributeNameSet();
if (!ours.equals(theirs))
return false;
for (String s : ours)
{
if (!Objects.equals(attributes.getAttribute(s), a.getAttribute(s)))
return false;
}
return true;
}
return false;
}
/** Unwrap attributes to a specific attribute {@link Wrapper}.
* @param attributes The attributes to unwrap, which may be a {@link Wrapper}
* @param target The target {@link Wrapper} class.
@ -203,20 +237,17 @@ public interface Attributes
getWrapped().clearAttributes();
}
// TODO: remove? or fix (don't want the wrapped and wrapper to match)
@Override
public int hashCode()
{
return getWrapped().hashCode();
return Attributes.hashCode(this);
}
// TODO: remove? or fix (don't want the wrapped and wrapper to match)
@Override
public boolean equals(Object obj)
public boolean equals(Object o)
{
return getWrapped().equals(obj);
return o instanceof Attributes && Attributes.equals(this, o);
}
}
/**
@ -240,8 +271,7 @@ public interface Attributes
public Mapped(Mapped attributes)
{
this();
_map.putAll(attributes._map);
this(new ConcurrentHashMap<>(attributes._map));
}
@Override
@ -300,33 +330,16 @@ public interface Attributes
setAttribute(name, attributes.getAttribute(name));
}
public Set<Map.Entry<String, Object>> getAttributeEntrySet()
{
return _map.entrySet();
}
// TODO: remove? or fix (don't want the wrapped and wrapper to match)
@Override
public int hashCode()
{
return _map.hashCode();
return Attributes.hashCode(this);
}
// TODO: remove? or fix (don't want the wrapped and wrapper to match)
@Override
public boolean equals(Object o)
{
if (o instanceof Attributes)
{
Attributes a = (Attributes)o;
for (Map.Entry<String, Object> e : _map.entrySet())
{
if (!Objects.equals(e.getValue(), a.getAttribute(e.getKey())))
return false;
}
return true;
}
return false;
return o instanceof Attributes && Attributes.equals(this, o);
}
}
@ -387,11 +400,6 @@ public interface Attributes
return map == null ? null : map.get(name);
}
public Collection<Object> getAttributeEntriesSet()
{
return map().values();
}
@Override
public Set<String> getAttributeNameSet()
{
@ -442,14 +450,25 @@ public interface Attributes
{
Dumpable.dumpObjects(out, indent, String.format("%s@%x", this.getClass().getSimpleName(), hashCode()), map());
}
@Override
public int hashCode()
{
return Attributes.hashCode(this);
}
@Override
public boolean equals(Object o)
{
return o instanceof Attributes && Attributes.equals(this, o);
}
}
/**
* An {@link Attributes} implementation backed by another {@link Attributes} instance, which is treated as immutable, but with a
* ConcurrentHashMap used as a mutable layer over it.
* // TODO: can we extend Wrapper with a contains wrapped?
*/
class Layer implements Attributes
class Layer extends Wrapper
{
private static final Object REMOVED = new Object()
{
@ -460,7 +479,6 @@ public interface Attributes
}
};
private final Attributes _persistent;
private final Attributes _layer;
public Layer(Attributes persistent)
@ -470,19 +488,19 @@ public interface Attributes
public Layer(Attributes persistent, Attributes layer)
{
_persistent = persistent;
super(persistent);
_layer = layer;
}
public Attributes getPersistentAttributes()
{
return _persistent;
return getWrapped();
}
@Override
public Object removeAttribute(String name)
{
Object p = _persistent.getAttribute(name);
Object p = super.getAttribute(name);
try
{
Object v = _layer.setAttribute(name, REMOVED);
@ -514,13 +532,13 @@ public interface Attributes
Object v = _layer.getAttribute(name);
if (v != null)
return v == REMOVED ? null : v;
return _persistent.getAttribute(name);
return super.getAttribute(name);
}
@Override
public Set<String> getAttributeNameSet()
{
Set<String> names = new HashSet<>(_persistent.getAttributeNameSet());
Set<String> names = new HashSet<>(super.getAttributeNameSet());
for (String name : _layer.getAttributeNameSet())
{
@ -536,40 +554,167 @@ public interface Attributes
public void clearAttributes()
{
_layer.clearAttributes();
for (String name : _persistent.getAttributeNameSet())
for (String name : super.getAttributeNameSet())
_layer.setAttribute(name, REMOVED);
}
}
// TODO: remove? or fix (don't want the wrapped and wrapper to match)
@Override
public int hashCode()
/**
* An abstract implementation of {@link Attributes.Wrapper} that provides a mechanism
* for synthetic attributes that can be modified or deleted. A synthetic attribute
* is one whose value is not stored in the normal map backing the {@link Attributes} instance,
* but is instead calculated as needed. Modifications to synthetic attributes are maintained
* in a separate layer and no modifications are made to the backing {@link Attributes}.
* <p>
* Non synthetic attributes are handled normally by the backing {@link Attributes}
* <p>
* Uses of this class must provide implementations for
* {@link #getSyntheticNameSet()} amd {@link #getSyntheticAttribute(String)}.
*/
abstract class Synthetic extends Wrapper
{
private static final Object REMOVED = new Object()
{
int hash = 113;
for (String name : getAttributeNameSet())
hash += name.hashCode() ^ getAttribute(name).hashCode();
return hash;
@Override
public String toString()
{
return "REMOVED";
}
};
private final AtomicReference<Map<String, Object>> _layer = new AtomicReference<>();
protected Synthetic(Attributes base)
{
super(base);
}
// TODO: remove? or fix (don't want the wrapped and wrapper to match)
@Override
public boolean equals(Object o)
{
if (o instanceof Attributes)
{
Attributes a = (Attributes)o;
Set<String> ours = getAttributeNameSet();
Set<String> theirs = getAttributeNameSet();
if (!ours.equals(theirs))
return false;
/**
* Get the value of a specific synthetic attribute.
* @param name The name of the attribute
* @return The value for the attribute, which may be computed on request, or {@code null}
*/
protected abstract Object getSyntheticAttribute(String name);
for (String s : ours)
{
if (!Objects.equals(getAttribute(s), a.getAttribute(s)))
return false;
}
return true;
/**
* Get the list of known synthetic attribute names, including those
* that currently have a null value.
* @return A {@link Set} of known synthetic attributes names.
*/
protected abstract Set<String> getSyntheticNameSet();
@Override
public Object getAttribute(String name)
{
// Has the attribute been modified in the layer?
Map<String, Object> layer = _layer.get();
if (layer != null)
{
// Only synthetic attributes can be in the layer.
Object l = layer.get(name);
// Has it been removed?
if (l == REMOVED)
return null;
// or has it been replaced?
if (l != null)
return l;
}
return false;
// Is there a synthetic value for the attribute? We just as for the value rather than checking the name.
Object s = getSyntheticAttribute(name);
if (s != null)
return s;
// otherwise get the attribute normally.
return super.getAttribute(name);
}
@Override
public Object setAttribute(String name, Object value)
{
// setting a null value is equivalent to removal
if (value == null)
return removeAttribute(name);
// is the attribute known to be synthetic?
if (getSyntheticNameSet().contains(name))
{
// We will need a layer to record modifications to a synthetic attribute
Map<String, Object> layer = _layer.updateAndGet(m -> m == null ? new HashMap<>() : m);
// update the attribute in the layer
Object old = layer.put(name, value);
// return the old value, which if not remove and not in the layer, is the synthetic attribute itself
return old == REMOVED ? null : old != null ? old : getSyntheticAttribute(name);
}
// handle non-synthetic attribute
return super.setAttribute(name, value);
}
@Override
public Object removeAttribute(String name)
{
// is the attribute known to be synthetic?
if (getSyntheticNameSet().contains(name))
{
// We will need a layer to record modifications to a synthetic attribute
Map<String, Object> layer = _layer.updateAndGet(m -> m == null ? new HashMap<>() : m);
// Mark the attribute as removed in the layer
Object old = layer.put(name, REMOVED);
// return the old value, which if not removed and not in the layer, is the synthetic attribute itself
return old == REMOVED ? null : old != null ? old : getSyntheticAttribute(name);
}
// handle non-synthetic attribute
return super.removeAttribute(name);
}
@Override
public Set<String> getAttributeNameSet()
{
// Get the non-synthetic attribute names
Set<String> names = new HashSet<>(super.getAttributeNameSet());
// Have there been any modifications to the synthetic attributes
Map<String, Object> layer = _layer.get();
if (layer == null)
{
// no, so we just add the names for which there are values
for (String s : getSyntheticNameSet())
if (getSyntheticAttribute(s) != null)
names.add(s);
}
else
{
// otherwise for each known synthetic name
for (String s : getSyntheticNameSet())
{
// has the attribute been modified in the layer?
Object l = layer.get(s);
if (l == REMOVED)
// it has been removed
names.remove(s);
else if (l != null || getSyntheticAttribute(s) != null)
// else it was modified or has an original value
names.add(s);
}
}
return Collections.unmodifiableSet(names);
}
@Override
public void clearAttributes()
{
// Clear the base attributes
super.clearAttributes();
// We will need a layer to remove the synthetic attributes
Map<String, Object> layer = _layer.updateAndGet(m -> m == null ? new HashMap<>() : m);
// remove all known synthetic attributes
for (String s : getSyntheticNameSet())
layer.put(s, REMOVED);
}
}
@ -609,5 +754,17 @@ public interface Attributes
{
return Collections.emptyMap();
}
@Override
public int hashCode()
{
return Attributes.hashCode(this);
}
@Override
public boolean equals(Object o)
{
return o instanceof Attributes && Attributes.equals(this, o);
}
};
}

View File

@ -49,6 +49,7 @@ import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Consumer;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import javax.net.ssl.CertPathTrustManagerParameters;
@ -113,6 +114,8 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
private static final Logger LOG = LoggerFactory.getLogger(SslContextFactory.class);
private static final Logger LOG_CONFIG = LoggerFactory.getLogger(LOG.getName() + ".config");
private static final Pattern KEY_SIZE_PATTERN = Pattern.compile("_(\\d+)_");
/**
* Default Excluded Protocols List
*/
@ -133,6 +136,7 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
"^.*_NULL_.*$",
"^.*_anon_.*$"
};
private static final String X_509 = "X.509";
private final AutoLock _lock = new AutoLock();
private final Set<String> _excludeProtocols = new LinkedHashSet<>();
@ -181,6 +185,7 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
private Factory _factory;
private PKIXCertPathChecker _pkixCertPathChecker;
private HostnameVerifier _hostnameVerifier;
private CertificateFactory _x509CertificateFactory;
/**
* Construct an instance of SslContextFactory with the default configuration.
@ -214,6 +219,7 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
{
load();
}
_x509CertificateFactory = getCertificateFactoryInstance(X_509);
checkConfiguration();
}
@ -302,7 +308,7 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
for (String alias : Collections.list(keyStore.aliases()))
{
Certificate certificate = keyStore.getCertificate(alias);
if (certificate != null && "X.509".equals(certificate.getType()))
if (certificate != null && X_509.equals(certificate.getType()))
{
X509Certificate x509C = (X509Certificate)certificate;
@ -1914,15 +1920,15 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
}
/**
* Obtain the X509 Certificate Chain from the provided SSLSession using the
* default {@link CertificateFactory} behaviors
* Obtain the X509 Certificate Chain from the provided SSLSession using this
* SslContextFactory's optional Provider specific {@link CertificateFactory}.
*
* @param sslSession the session to use for active peer certificates
* @return the certificate chain
*/
public static X509Certificate[] getCertChain(SSLSession sslSession)
public X509Certificate[] getX509CertChain(SSLSession sslSession)
{
return getX509CertChain(null, sslSession);
return getX509CertChain(_x509CertificateFactory, sslSession);
}
/**
@ -1932,15 +1938,17 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
* @param sslSession the session to use for active peer certificates
* @return the certificate chain
*/
public X509Certificate[] getX509CertChain(SSLSession sslSession)
public static X509Certificate[] getCertChain(SSLSession sslSession)
{
return getX509CertChain(this, sslSession);
return getX509CertChain(null, sslSession);
}
private static X509Certificate[] getX509CertChain(SslContextFactory sslContextFactory, SSLSession sslSession)
private static X509Certificate[] getX509CertChain(CertificateFactory certificateFactory, SSLSession sslSession)
{
try
{
if (certificateFactory == null)
certificateFactory = CertificateFactory.getInstance(X_509);
Certificate[] javaxCerts = sslSession.getPeerCertificates();
if (javaxCerts == null || javaxCerts.length == 0)
return null;
@ -1948,22 +1956,11 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
int length = javaxCerts.length;
X509Certificate[] javaCerts = new X509Certificate[length];
String type = "X.509";
CertificateFactory cf;
if (sslContextFactory != null)
{
cf = sslContextFactory.getCertificateFactoryInstance(type);
}
else
{
cf = CertificateFactory.getInstance(type);
}
for (int i = 0; i < length; i++)
{
byte[] bytes = javaxCerts[i].getEncoded();
ByteArrayInputStream stream = new ByteArrayInputStream(bytes);
javaCerts[i] = (X509Certificate)cf.generateCertificate(stream);
javaCerts[i] = (X509Certificate)certificateFactory.generateCertificate(stream);
}
return javaCerts;
@ -1999,6 +1996,8 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
* DES_CBC Block 56
* 3DES_EDE_CBC Block 168
* </pre>
* <p>
* For unknown ciphers, any substring of digits bounded by '_' is taken as the key length.
*
* @param cipherSuite String name of the TLS cipher suite.
* @return int indicating the effective key entropy bit-length.
@ -2008,26 +2007,30 @@ public abstract class SslContextFactory extends ContainerLifeCycle implements Du
// Roughly ordered from most common to least common.
if (cipherSuite == null)
return 0;
else if (cipherSuite.contains("WITH_AES_256_"))
return 256;
else if (cipherSuite.contains("WITH_RC4_128_"))
return 128;
else if (cipherSuite.contains("WITH_AES_128_"))
return 128;
else if (cipherSuite.contains("WITH_RC4_40_"))
return 40;
else if (cipherSuite.contains("WITH_3DES_EDE_CBC_"))
if (cipherSuite.contains("WITH_3DES_EDE_CBC_"))
return 168;
else if (cipherSuite.contains("WITH_IDEA_CBC_"))
if (cipherSuite.contains("WITH_IDEA_CBC_"))
return 128;
else if (cipherSuite.contains("WITH_RC2_CBC_40_"))
return 40;
else if (cipherSuite.contains("WITH_DES40_CBC_"))
return 40;
else if (cipherSuite.contains("WITH_DES_CBC_"))
return 56;
else
return 0;
Matcher matcher = KEY_SIZE_PATTERN.matcher(cipherSuite);
if (matcher.find())
{
String keyLengthString = matcher.group(1);
try
{
return Integer.parseInt(keyLengthString);
}
catch (NumberFormatException e)
{
if (LOG.isTraceEnabled())
LOG.trace("unknown key length", e);
}
}
return 0;
}
public void validateCerts(X509Certificate[] certs) throws Exception

View File

@ -13,6 +13,8 @@
package org.eclipse.jetty.util;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Stream;
import org.junit.jupiter.api.Test;
@ -34,6 +36,7 @@ public class AttributesTest
new Attributes.Mapped(),
new Attributes.Wrapper(new Attributes.Mapped()),
new Attributes.Layer(new Attributes.Mapped()),
new TestSynthetic(new Attributes.Mapped()),
new AttributesMap()
);
}
@ -135,4 +138,84 @@ public class AttributesTest
testAttributes(layer);
}
@Test
public void testSynthetic()
{
TestSynthetic attributes = new TestSynthetic(new Attributes.Mapped());
assertThat(attributes.getAttributeNameSet(), empty());
attributes._roy.set("Batty");
attributes._leon.set("Kowalski");
assertThat(attributes.getAttribute("Leon"), equalTo("Kowalski"));
assertThat(attributes.getAttribute("Zhora"), nullValue());
assertThat(attributes.getAttribute("Pris"), nullValue());
assertThat(attributes.getAttribute("Roy"), equalTo("Batty"));
assertThat(attributes.getAttribute("A"), nullValue());
assertThat(attributes.getAttributeNameSet(), containsInAnyOrder("Roy", "Leon"));
assertThat(attributes.setAttribute("A", "1"), nullValue());
assertThat(attributes.setAttribute("Pris", "Unknown"), nullValue());
assertThat(attributes.getAttribute("Leon"), equalTo("Kowalski"));
assertThat(attributes.getAttribute("Pris"), equalTo("Unknown"));
assertThat(attributes.getAttribute("Roy"), equalTo("Batty"));
assertThat(attributes.getAttribute("A"), equalTo("1"));
assertThat(attributes.getAttributeNameSet(), containsInAnyOrder("Roy", "Leon", "A", "Pris"));
assertThat(attributes.setAttribute("Leon", "retired"), equalTo("Kowalski"));
assertThat(attributes.setAttribute("Zhora", "retired"), nullValue());
assertThat(attributes.setAttribute("A", "2"), equalTo("1"));
assertThat(attributes.getAttribute("Leon"), equalTo("retired"));
assertThat(attributes.getAttribute("Zhora"), equalTo("retired"));
assertThat(attributes.getAttribute("Pris"), equalTo("Unknown"));
assertThat(attributes.getAttribute("Roy"), equalTo("Batty"));
assertThat(attributes.getAttribute("A"), equalTo("2"));
assertThat(attributes.getAttributeNameSet(), containsInAnyOrder("Roy", "Leon", "A", "Pris", "Zhora"));
assertThat(attributes.removeAttribute("Leon"), equalTo("retired"));
assertThat(attributes.removeAttribute("Zhora"), equalTo("retired"));
assertThat(attributes.removeAttribute("Pris"), equalTo("Unknown"));
assertThat(attributes.getAttribute("Roy"), equalTo("Batty"));
assertThat(attributes.getAttribute("A"), equalTo("2"));
assertThat(attributes.getAttributeNameSet(), containsInAnyOrder("Roy", "A"));
attributes.clearAttributes();
assertThat(attributes.getAttributeNameSet(), empty());
}
private static class TestSynthetic extends Attributes.Synthetic
{
Set<String> _syntheticAttributes = Set.of("Roy", "Pris", "Zhora", "Leon");
AtomicReference<Object> _roy = new AtomicReference<>();
AtomicReference<Object> _pris = new AtomicReference<>();
AtomicReference<Object> _zhora = new AtomicReference<>();
AtomicReference<Object> _leon = new AtomicReference<>();
public TestSynthetic(Attributes attributes)
{
super(attributes);
}
@Override
protected Object getSyntheticAttribute(String name)
{
return switch (name)
{
case "Roy" -> _roy.get();
case "Pris" -> _pris.get();
case "Zhora" -> _zhora.get();
case "Leon" -> _leon.get();
default -> null;
};
}
@Override
protected Set<String> getSyntheticNameSet()
{
return _syntheticAttributes;
}
}
}

View File

@ -39,6 +39,7 @@ import org.eclipse.jetty.client.HttpClientTransport;
import org.eclipse.jetty.client.Request;
import org.eclipse.jetty.client.transport.HttpClientTransportDynamic;
import org.eclipse.jetty.ee10.servlet.ServletContextHandler;
import org.eclipse.jetty.ee10.servlet.ServletContextRequest;
import org.eclipse.jetty.ee10.servlet.ServletHolder;
import org.eclipse.jetty.http.HttpScheme;
import org.eclipse.jetty.http.HttpStatus;
@ -46,6 +47,7 @@ import org.eclipse.jetty.io.ClientConnectionFactory;
import org.eclipse.jetty.io.ClientConnector;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.ssl.SslClientConnectionFactory;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.HttpConfiguration;
@ -140,13 +142,17 @@ public class ClientAuthProxyTest
@Override
public boolean handle(org.eclipse.jetty.server.Request request, Response response, Callback callback)
{
X509Certificate[] certificates = (X509Certificate[])request.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE);
Assertions.assertNotNull(certificates);
X509Certificate certificate = certificates[0];
X500Principal principal = certificate.getSubjectX500Principal();
String body = "%s\r\n%d\r\n".formatted(principal.toString(), org.eclipse.jetty.server.Request.getRemotePort(request));
Content.Sink.write(response, true, body, callback);
return true;
if (request.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData)
{
X509Certificate[] certificates = sslSessionData.peerCertificates();
Assertions.assertNotNull(certificates);
X509Certificate certificate = certificates[0];
X500Principal principal = certificate.getSubjectX500Principal();
String body = "%s\r\n%d\r\n".formatted(principal.toString(), org.eclipse.jetty.server.Request.getRemotePort(request));
Content.Sink.write(response, true, body, callback);
return true;
}
return false;
}
});
}
@ -213,7 +219,7 @@ public class ClientAuthProxyTest
private static String retrieveUser(HttpServletRequest request)
{
X509Certificate[] certificates = (X509Certificate[])request.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE);
X509Certificate[] certificates = (X509Certificate[])request.getAttribute(ServletContextRequest.PEER_CERTIFICATES);
String clientName = certificates[0].getSubjectX500Principal().getName();
Matcher matcher = Pattern.compile("CN=([^,]+)").matcher(clientName);
if (matcher.find())
@ -467,9 +473,9 @@ public class ClientAuthProxyTest
@Override
protected void service(HttpServletRequest request, HttpServletResponse response)
{
Assertions.assertNotNull(request.getAttribute("jakarta.servlet.request.cipher_suite"));
Assertions.assertNotNull(request.getAttribute("jakarta.servlet.request.key_size"));
Assertions.assertNotNull(request.getAttribute("jakarta.servlet.request.ssl_session_id"));
Assertions.assertNotNull(request.getAttribute(ServletContextRequest.SSL_CIPHER_SUITE));
Assertions.assertNotNull(request.getAttribute(ServletContextRequest.SSL_KEY_SIZE));
Assertions.assertNotNull(request.getAttribute(ServletContextRequest.SSL_SESSION_ID));
Assertions.assertNotNull(request.getAttribute("jakarta.servlet.request.X509Certificate"));
// Do not proxy these requests, we just want to verify the attributes.
response.setStatus(HttpStatus.OK_200);

View File

@ -17,10 +17,8 @@ import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EventListener;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.function.Supplier;
import jakarta.servlet.AsyncListener;
import jakarta.servlet.ServletRequest;
@ -33,16 +31,17 @@ import org.eclipse.jetty.http.HttpMethod;
import org.eclipse.jetty.http.HttpURI;
import org.eclipse.jetty.http.UriCompliance;
import org.eclipse.jetty.http.pathmap.MatchedResource;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.server.FormFields;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.Response;
import org.eclipse.jetty.server.SecureRequestCustomizer;
import org.eclipse.jetty.server.Session;
import org.eclipse.jetty.server.handler.ContextHandler;
import org.eclipse.jetty.server.handler.ContextRequest;
import org.eclipse.jetty.session.AbstractSessionManager;
import org.eclipse.jetty.session.ManagedSession;
import org.eclipse.jetty.session.SessionManager;
import org.eclipse.jetty.util.Attributes;
import org.eclipse.jetty.util.Fields;
import org.eclipse.jetty.util.URIUtil;
@ -58,6 +57,20 @@ import org.eclipse.jetty.util.URIUtil;
public class ServletContextRequest extends ContextRequest implements ServletContextHandler.ServletRequestInfo, Request.ServeAs
{
public static final String MULTIPART_CONFIG_ELEMENT = "org.eclipse.jetty.multipartConfig";
public static final String SSL_CIPHER_SUITE = "jakarta.servlet.request.cipher_suite";
public static final String SSL_KEY_SIZE = "jakarta.servlet.request.key_size";
public static final String SSL_SESSION_ID = "jakarta.servlet.request.ssl_session_id";
public static final String PEER_CERTIFICATES = "jakarta.servlet.request.X509Certificate";
private static final Set<String> ATTRIBUTES = Set.of(
SSL_CIPHER_SUITE,
SSL_KEY_SIZE,
SSL_SESSION_ID,
PEER_CERTIFICATES,
MULTIPART_CONFIG_ELEMENT,
FormFields.MAX_FIELDS_ATTRIBUTE,
FormFields.MAX_LENGTH_ATTRIBUTE);
static final int INPUT_NONE = 0;
static final int INPUT_STREAM = 1;
static final int INPUT_READER = 2;
@ -65,8 +78,6 @@ public class ServletContextRequest extends ContextRequest implements ServletCont
static final Fields NO_PARAMS = new Fields(Collections.emptyMap());
static final Fields BAD_PARAMS = new Fields(Collections.emptyMap());
private static final Object NULL_VALUE = new Object();
public static ServletContextRequest getServletContextRequest(ServletRequest request)
{
if (request instanceof ServletApiRequest servletApiRequest &&
@ -95,6 +106,8 @@ public class ServletContextRequest extends ContextRequest implements ServletCont
private final String _decodedPathInContext;
private final ServletChannel _servletChannel;
private final SessionManager _sessionManager;
private final Attributes _attributes;
private List<ServletRequestAttributeListener> _requestAttributeListeners;
private Charset _queryEncoding;
private HttpFields _trailers;
@ -118,6 +131,30 @@ public class ServletContextRequest extends ContextRequest implements ServletCont
_decodedPathInContext = decodedPathInContext;
_response = newServletContextResponse(response);
_sessionManager = sessionManager;
_attributes = new Attributes.Synthetic(request)
{
@Override
protected Object getSyntheticAttribute(String name)
{
return switch (name)
{
case SSL_CIPHER_SUITE -> super.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData data ? data.cipherSuite() : null;
case SSL_KEY_SIZE -> super.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData data ? data.keySize() : null;
case SSL_SESSION_ID -> super.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData data ? data.sslSessionId() : null;
case PEER_CERTIFICATES -> super.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData data ? data.peerCertificates() : null;
case ServletContextRequest.MULTIPART_CONFIG_ELEMENT -> _matchedResource.getResource().getServletHolder().getMultipartConfigElement();
case FormFields.MAX_FIELDS_ATTRIBUTE -> getServletContext().getServletContextHandler().getMaxFormKeys();
case FormFields.MAX_LENGTH_ATTRIBUTE -> getServletContext().getServletContextHandler().getMaxFormContentSize();
default -> null;
};
}
@Override
protected Set<String> getSyntheticNameSet()
{
return ATTRIBUTES;
}
};
addIdleTimeoutListener(_servletChannel.getServletRequestState()::onIdleTimeout);
}
@ -264,88 +301,28 @@ public class ServletContextRequest extends ContextRequest implements ServletCont
return _queryEncoding;
}
private Object getAttributeNotNullOrElse(String name, Supplier<Object> getter)
{
Object value = super.getAttribute(name);
if (value == NULL_VALUE)
return null;
if (value != null)
return value;
return getter.get();
}
@Override
public Object getAttribute(String name)
{
return switch (name)
{
case "jakarta.servlet.request.cipher_suite" -> super.getAttribute(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE);
case "jakarta.servlet.request.key_size" -> super.getAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE);
case "jakarta.servlet.request.ssl_session_id" -> super.getAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE);
case "jakarta.servlet.request.X509Certificate" -> super.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE);
case ServletContextRequest.MULTIPART_CONFIG_ELEMENT -> getAttributeNotNullOrElse(name, _matchedResource.getResource().getServletHolder()::getMultipartConfigElement);
case FormFields.MAX_FIELDS_ATTRIBUTE -> getAttributeNotNullOrElse(name, getServletContext().getServletContextHandler()::getMaxFormKeys);
case FormFields.MAX_LENGTH_ATTRIBUTE -> getAttributeNotNullOrElse(name, getServletContext().getServletContextHandler()::getMaxFormContentSize);
default -> super.getAttribute(name);
};
return _attributes.getAttribute(name);
}
@Override
public Object removeAttribute(String name)
{
return switch (name)
{
case "jakarta.servlet.request.cipher_suite" -> super.removeAttribute(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE);
case "jakarta.servlet.request.key_size" -> super.removeAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE);
case "jakarta.servlet.request.ssl_session_id" -> super.removeAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE);
case "jakarta.servlet.request.X509Certificate" -> super.removeAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE);
case ServletContextRequest.MULTIPART_CONFIG_ELEMENT, FormFields.MAX_FIELDS_ATTRIBUTE, FormFields.MAX_LENGTH_ATTRIBUTE -> super.setAttribute(name, NULL_VALUE);
default -> super.removeAttribute(name);
};
return _attributes.removeAttribute(name);
}
@Override
public Object setAttribute(String name, Object value)
{
if (value == null)
return removeAttribute(name);
return switch (name)
{
case "jakarta.servlet.request.cipher_suite" -> super.setAttribute(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE, value);
case "jakarta.servlet.request.key_size" -> super.setAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE, value);
case "jakarta.servlet.request.ssl_session_id" -> super.setAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE, value);
case "jakarta.servlet.request.X509Certificate" -> super.setAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE, value);
default -> super.setAttribute(name, value);
};
}
private void checkContainsNotNull(Set<String> names, String name, Supplier<Boolean> contains)
{
Object value = super.getAttribute(name);
if (value == NULL_VALUE)
names.remove(name);
else if (value != null || contains.get())
names.add(name);
return _attributes.setAttribute(name, value);
}
@Override
public Set<String> getAttributeNameSet()
{
Set<String> names = new HashSet<>(super.getAttributeNameSet());
if (names.contains(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE))
names.add("jakarta.servlet.request.cipher_suite");
if (names.contains(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE))
names.add("jakarta.servlet.request.key_size");
if (names.contains(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE))
names.add("jakarta.servlet.request.ssl_session_id");
if (names.contains(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE))
names.add("jakarta.servlet.request.X509Certificate");
checkContainsNotNull(names, ServletContextRequest.MULTIPART_CONFIG_ELEMENT, () -> _matchedResource.getResource().getServletHolder().getMultipartConfigElement() != null);
checkContainsNotNull(names, FormFields.MAX_FIELDS_ATTRIBUTE, () -> getServletContext().getServletContextHandler().getMaxFormKeys() >= 0);
checkContainsNotNull(names, FormFields.MAX_LENGTH_ATTRIBUTE, () -> getServletContext().getServletContextHandler().getMaxFormContentSize() >= 0L);
return names;
return _attributes.getAttributeNameSet();
}
/**

View File

@ -15,6 +15,7 @@ package org.eclipse.jetty.ee10.servlet;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -32,12 +33,12 @@ import jakarta.servlet.http.HttpServletResponse;
import org.eclipse.jetty.http.HttpStatus;
import org.eclipse.jetty.http.HttpTester;
import org.eclipse.jetty.http.UriCompliance;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.server.FormFields;
import org.eclipse.jetty.server.ForwardedRequestCustomizer;
import org.eclipse.jetty.server.HttpConnectionFactory;
import org.eclipse.jetty.server.LocalConnector;
import org.eclipse.jetty.server.Request;
import org.eclipse.jetty.server.SecureRequestCustomizer;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.util.component.LifeCycle;
import org.junit.jupiter.api.AfterEach;
@ -335,37 +336,28 @@ public class RequestTest
Request coreRequest = servletContextRequest.getRequest();
// Set some fake SSL attributes
Object certificate = new Object();
coreRequest.setAttribute(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE, "quantumKnowledge");
coreRequest.setAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE, 42);
coreRequest.setAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE, "identity");
coreRequest.setAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE, certificate);
X509Certificate[] certificates = new X509Certificate[0];
coreRequest.setAttribute(EndPoint.SslSessionData.ATTRIBUTE,
EndPoint.SslSessionData.from(null, "identity", "quantum_42_Knowledge", certificates));
// Check we have all the attribute names in servlet API
Set<String> names = new HashSet<>(Collections.list(request.getAttributeNames()));
assertThat(names, containsInAnyOrder(
SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE,
"jakarta.servlet.request.cipher_suite",
SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE,
"jakarta.servlet.request.key_size",
SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE,
"jakarta.servlet.request.ssl_session_id",
SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE,
"jakarta.servlet.request.X509Certificate",
EndPoint.SslSessionData.ATTRIBUTE,
ServletContextRequest.SSL_CIPHER_SUITE,
ServletContextRequest.SSL_KEY_SIZE,
ServletContextRequest.SSL_SESSION_ID,
ServletContextRequest.PEER_CERTIFICATES,
FormFields.MAX_FIELDS_ATTRIBUTE,
FormFields.MAX_LENGTH_ATTRIBUTE,
ServletContextRequest.MULTIPART_CONFIG_ELEMENT
));
// check we can get the expected values
assertThat(request.getAttribute(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE), is("quantumKnowledge"));
assertThat(request.getAttribute("jakarta.servlet.request.cipher_suite"), is("quantumKnowledge"));
assertThat(request.getAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE), is(42));
assertThat(request.getAttribute("jakarta.servlet.request.key_size"), is(42));
assertThat(request.getAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE), is("identity"));
assertThat(request.getAttribute("jakarta.servlet.request.ssl_session_id"), is("identity"));
assertThat(request.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE), sameInstance(certificate));
assertThat(request.getAttribute("jakarta.servlet.request.X509Certificate"), sameInstance(certificate));
assertThat(request.getAttribute(ServletContextRequest.SSL_CIPHER_SUITE), is("quantum_42_Knowledge"));
assertThat(request.getAttribute(ServletContextRequest.SSL_KEY_SIZE), is(42));
assertThat(request.getAttribute(ServletContextRequest.SSL_SESSION_ID), is("identity"));
assertThat(request.getAttribute(ServletContextRequest.PEER_CERTIFICATES), sameInstance(certificates));
assertThat(request.getAttribute(ServletContextRequest.MULTIPART_CONFIG_ELEMENT), notNullValue());
int maxFormKeys = ServletContextHandler.getServletContextHandler(request.getServletContext()).getMaxFormKeys();
assertThat(request.getAttribute(FormFields.MAX_FIELDS_ATTRIBUTE), is(maxFormKeys));
@ -373,23 +365,19 @@ public class RequestTest
assertThat(request.getAttribute(FormFields.MAX_LENGTH_ATTRIBUTE), is(maxFormContentSize));
// check we can set all those attributes in the servlet API
request.setAttribute("jakarta.servlet.request.cipher_suite", "piglatin");
request.setAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE, 3);
request.setAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE, "other");
request.setAttribute("jakarta.servlet.request.X509Certificate", "certificate");
request.setAttribute(ServletContextRequest.SSL_CIPHER_SUITE, "pig_33_latin");
request.setAttribute(ServletContextRequest.SSL_KEY_SIZE, 3);
request.setAttribute(ServletContextRequest.SSL_SESSION_ID, "other");
request.setAttribute(ServletContextRequest.PEER_CERTIFICATES, "certificate");
request.setAttribute(ServletContextRequest.MULTIPART_CONFIG_ELEMENT, "config2");
request.setAttribute(FormFields.MAX_FIELDS_ATTRIBUTE, 101);
request.setAttribute(FormFields.MAX_LENGTH_ATTRIBUTE, 102);
// check we can get the updated values
assertThat(request.getAttribute(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE), is("piglatin"));
assertThat(request.getAttribute("jakarta.servlet.request.cipher_suite"), is("piglatin"));
assertThat(request.getAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE), is(3));
assertThat(request.getAttribute("jakarta.servlet.request.key_size"), is(3));
assertThat(request.getAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE), is("other"));
assertThat(request.getAttribute("jakarta.servlet.request.ssl_session_id"), is("other"));
assertThat(request.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE), is("certificate"));
assertThat(request.getAttribute("jakarta.servlet.request.X509Certificate"), is("certificate"));
assertThat(request.getAttribute(ServletContextRequest.SSL_CIPHER_SUITE), is("pig_33_latin"));
assertThat(request.getAttribute(ServletContextRequest.SSL_KEY_SIZE), is(3));
assertThat(request.getAttribute(ServletContextRequest.SSL_SESSION_ID), is("other"));
assertThat(request.getAttribute(ServletContextRequest.PEER_CERTIFICATES), is("certificate"));
assertThat(request.getAttribute(ServletContextRequest.MULTIPART_CONFIG_ELEMENT), is("config2"));
assertThat(request.getAttribute(FormFields.MAX_FIELDS_ATTRIBUTE), is(101));
assertThat(request.getAttribute(FormFields.MAX_LENGTH_ATTRIBUTE), is(102));
@ -400,13 +388,14 @@ public class RequestTest
assertThat(ServletContextHandler.getServletContextHandler(request.getServletContext()).getMaxFormContentSize(), is(maxFormContentSize));
// Check we can remove all the attributes
request.removeAttribute("jakarta.servlet.request.cipher_suite");
request.removeAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE);
request.setAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE, null);
request.setAttribute("jakarta.servlet.request.X509Certificate", null);
request.removeAttribute(ServletContextRequest.SSL_CIPHER_SUITE);
request.removeAttribute(ServletContextRequest.SSL_KEY_SIZE);
request.removeAttribute(ServletContextRequest.SSL_SESSION_ID);
request.setAttribute(ServletContextRequest.PEER_CERTIFICATES, null);
request.removeAttribute(ServletContextRequest.MULTIPART_CONFIG_ELEMENT);
request.removeAttribute(FormFields.MAX_FIELDS_ATTRIBUTE);
request.removeAttribute(FormFields.MAX_LENGTH_ATTRIBUTE);
request.removeAttribute(EndPoint.SslSessionData.ATTRIBUTE);
assertThat(Collections.list(request.getAttributeNames()), empty());

View File

@ -38,7 +38,7 @@ import org.eclipse.jetty.util.ssl.SslContextFactory;
public class HTTP1Servlet extends HttpServlet
{
private SslContextFactory sslContextFactory;
private SslContextFactory.Client sslContextFactory;
private HTTP2Client http2Client;
@Override

View File

@ -1,102 +0,0 @@
//
// ========================================================================
// Copyright (c) 1995 Mort Bay Consulting Pty Ltd and others.
//
// This program and the accompanying materials are made available under the
// terms of the Eclipse Public License v. 2.0 which is available at
// https://www.eclipse.org/legal/epl-2.0, or the Apache License, Version 2.0
// which is available at https://www.apache.org/licenses/LICENSE-2.0.
//
// SPDX-License-Identifier: EPL-2.0 OR Apache-2.0
// ========================================================================
//
package org.eclipse.jetty.ee9.nested;
import java.util.HashSet;
import java.util.Set;
import jakarta.servlet.AsyncContext;
import org.eclipse.jetty.util.Attributes;
class AsyncAttributes extends Attributes.Wrapper
{
private final String _requestURI;
private final String _contextPath;
private final ServletPathMapping _mapping;
private final String _queryString;
private final String _servletPath;
private final String _pathInfo;
public AsyncAttributes(Attributes attributes, String requestUri, String contextPath, String pathInContext, ServletPathMapping mapping, String queryString)
{
super(attributes);
_requestURI = requestUri;
_contextPath = contextPath;
_servletPath = mapping == null ? null : mapping.getServletPath();
_pathInfo = mapping == null ? pathInContext : mapping.getPathInfo();
_mapping = mapping;
_queryString = queryString;
}
@Override
public Object getAttribute(String key)
{
switch (key)
{
case AsyncContext.ASYNC_REQUEST_URI:
return _requestURI;
case AsyncContext.ASYNC_CONTEXT_PATH:
return _contextPath;
case AsyncContext.ASYNC_SERVLET_PATH:
return _servletPath;
case AsyncContext.ASYNC_PATH_INFO:
return _pathInfo;
case AsyncContext.ASYNC_QUERY_STRING:
return _queryString;
case AsyncContext.ASYNC_MAPPING:
return _mapping;
default:
return super.getAttribute(key);
}
}
@Override
public Set<String> getAttributeNameSet()
{
Set<String> set = new HashSet<>(super.getAttributeNameSet());
if (_requestURI != null)
set.add(AsyncContext.ASYNC_REQUEST_URI);
if (_contextPath != null)
set.add(AsyncContext.ASYNC_CONTEXT_PATH);
if (_servletPath != null)
set.add(AsyncContext.ASYNC_SERVLET_PATH);
if (_pathInfo != null)
set.add(AsyncContext.ASYNC_PATH_INFO);
if (_queryString != null)
set.add(AsyncContext.ASYNC_QUERY_STRING);
if (_mapping != null)
set.add(AsyncContext.ASYNC_MAPPING);
return set;
}
@Override
public Object setAttribute(String key, Object value)
{
switch (key)
{
case AsyncContext.ASYNC_REQUEST_URI:
case AsyncContext.ASYNC_CONTEXT_PATH:
case AsyncContext.ASYNC_SERVLET_PATH:
case AsyncContext.ASYNC_PATH_INFO:
case AsyncContext.ASYNC_QUERY_STRING:
case AsyncContext.ASYNC_MAPPING:
// Ignore sets for these reserved names as this class is applied
// we will always override these particular attributes.
return null;
default:
return super.setAttribute(key, value);
}
}
}

View File

@ -104,7 +104,21 @@ import org.slf4j.LoggerFactory;
*/
public class Request implements HttpServletRequest
{
public static final String __MULTIPART_CONFIG_ELEMENT = "org.eclipse.jetty.multipartConfig";
/**
* The name of the MultiPartConfig request attribute
*/
public static final String MULTIPART_CONFIG_ELEMENT = "org.eclipse.jetty.multipartConfig";
/**
* @deprecated use {@link #MULTIPART_CONFIG_ELEMENT}
*/
@Deprecated
public static final String __MULTIPART_CONFIG_ELEMENT = MULTIPART_CONFIG_ELEMENT;
public static final String SSL_CIPHER_SUITE = "jakarta.servlet.request.cipher_suite";
public static final String SSL_KEY_SIZE = "jakarta.servlet.request.key_size";
public static final String SSL_SESSION_ID = "jakarta.servlet.request.ssl_session_id";
public static final String PEER_CERTIFICATES = "jakarta.servlet.request.X509Certificate";
private static final Logger LOG = LoggerFactory.getLogger(Request.class);
private static final SetCookieParser SET_COOKIE_PARSER = SetCookieParser.newInstance();
@ -438,7 +452,7 @@ public class Request implements HttpServletRequest
extractFormParameters(_contentParameters);
}
else if (MimeTypes.Type.MULTIPART_FORM_DATA.is(baseType) &&
getAttribute(__MULTIPART_CONFIG_ELEMENT) != null &&
getAttribute(MULTIPART_CONFIG_ELEMENT) != null &&
_multiParts == null)
{
try
@ -1892,7 +1906,7 @@ public class Request implements HttpServletRequest
{
if (_multiParts == null)
{
MultipartConfigElement config = (MultipartConfigElement)getAttribute(__MULTIPART_CONFIG_ELEMENT);
MultipartConfigElement config = (MultipartConfigElement)getAttribute(MULTIPART_CONFIG_ELEMENT);
if (config == null)
throw new IllegalStateException("No multipart config for servlet");

View File

@ -13,104 +13,74 @@
package org.eclipse.jetty.ee9.nested;
import java.util.HashSet;
import java.util.Set;
import org.eclipse.jetty.server.SecureRequestCustomizer;
import jakarta.servlet.AsyncContext;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.util.Attributes;
import org.eclipse.jetty.util.AttributesMap;
/**
* An implementation of Attributes that supports the standard async attributes.
*
* This implementation delegates to an internal {@link AttributesMap} instance, which
* can optionally be wrapped with a {@link AsyncAttributes} instance. This allows async
* attributes to be applied underneath any other attribute wrappers.
* An implementation of Attributes that supports the standard SSL and async attributes.
*/
public class ServletAttributes implements Attributes
public class ServletAttributes extends Attributes.Synthetic
{
private final Attributes _attributes;
private AsyncAttributes _asyncAttributes;
private static final Set<String> ATTRIBUTES =
Set.of(
Request.SSL_CIPHER_SUITE,
Request.SSL_KEY_SIZE,
Request.SSL_SESSION_ID,
Request.PEER_CERTIFICATES,
AsyncContext.ASYNC_REQUEST_URI,
AsyncContext.ASYNC_CONTEXT_PATH,
AsyncContext.ASYNC_SERVLET_PATH,
AsyncContext.ASYNC_PATH_INFO,
AsyncContext.ASYNC_QUERY_STRING,
AsyncContext.ASYNC_MAPPING
);
private record Async(
String requestURI,
String contextPath,
String pathInContext,
ServletPathMapping mapping,
String queryString)
{
}
private Async _async;
ServletAttributes(Attributes attributes)
{
_attributes = new SSLAttributes(attributes);
super(attributes);
}
@Override
protected Object getSyntheticAttribute(String name)
{
return switch (name)
{
case Request.SSL_CIPHER_SUITE -> getWrapped().getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData ? sslSessionData.cipherSuite() : null;
case Request.SSL_KEY_SIZE -> getWrapped().getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData ? sslSessionData.keySize() : null;
case Request.SSL_SESSION_ID -> getWrapped().getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData ? sslSessionData.sslSessionId() : null;
case Request.PEER_CERTIFICATES -> getWrapped().getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData ? sslSessionData.peerCertificates() : null;
case AsyncContext.ASYNC_REQUEST_URI -> _async == null ? null : _async.requestURI;
case AsyncContext.ASYNC_CONTEXT_PATH -> _async == null ? null : _async.contextPath;
case AsyncContext.ASYNC_SERVLET_PATH -> _async == null ? null : _async.mapping == null ? null : _async.mapping.getServletPath();
case AsyncContext.ASYNC_PATH_INFO -> _async == null ? null : _async.mapping == null ? _async.pathInContext : _async.mapping.getPathInfo();
case AsyncContext.ASYNC_QUERY_STRING -> _async == null ? null : _async.queryString;
case AsyncContext.ASYNC_MAPPING -> _async == null ? null : _async.mapping;
default -> null;
};
}
@Override
protected Set<String> getSyntheticNameSet()
{
return ATTRIBUTES;
}
public void setAsyncAttributes(String requestURI, String contextPath, String pathInContext, ServletPathMapping servletPathMapping, String queryString)
{
_asyncAttributes = new AsyncAttributes(_attributes, requestURI, contextPath, pathInContext, servletPathMapping, queryString);
}
private Attributes getAttributes()
{
return (_asyncAttributes == null) ? _attributes : _asyncAttributes;
}
@Override
public Object removeAttribute(String name)
{
return getAttributes().removeAttribute(name);
}
@Override
public Object setAttribute(String name, Object attribute)
{
return getAttributes().setAttribute(name, attribute);
}
@Override
public Object getAttribute(String name)
{
return getAttributes().getAttribute(name);
}
@Override
public Set<String> getAttributeNameSet()
{
return getAttributes().getAttributeNameSet();
}
@Override
public void clearAttributes()
{
getAttributes().clearAttributes();
_asyncAttributes = null;
}
private static class SSLAttributes extends Attributes.Wrapper
{
public SSLAttributes(Attributes wrapped)
{
super(wrapped);
}
@Override
public Object getAttribute(String name)
{
return switch (name)
{
case "jakarta.servlet.request.cipher_suite" -> super.getAttribute(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE);
case "jakarta.servlet.request.key_size" -> super.getAttribute(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE);
case "jakarta.servlet.request.ssl_session_id" -> super.getAttribute(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE);
case "jakarta.servlet.request.X509Certificate" -> super.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE);
default -> super.getAttribute(name);
};
}
@Override
public Set<String> getAttributeNameSet()
{
Set<String> names = new HashSet<>(super.getAttributeNameSet());
if (names.contains(SecureRequestCustomizer.CIPHER_SUITE_ATTRIBUTE))
names.add("jakarta.servlet.request.cipher_suite");
if (names.contains(SecureRequestCustomizer.KEY_SIZE_ATTRIBUTE))
names.add("jakarta.servlet.request.key_size");
if (names.contains(SecureRequestCustomizer.SSL_SESSION_ID_ATTRIBUTE))
names.add("jakarta.servlet.request.ssl_session_id");
if (names.contains(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE))
names.add("jakarta.servlet.request.X509Certificate");
return names;
}
_async = new Async(requestURI, contextPath, pathInContext, servletPathMapping, queryString);
}
}

View File

@ -2263,7 +2263,7 @@ public class RequestTest
try
{
MultipartConfigElement mpce = new MultipartConfigElement(tmpDir.getAbsolutePath(), -1, -1, 2);
request.setAttribute(Request.__MULTIPART_CONFIG_ELEMENT, mpce);
request.setAttribute(Request.MULTIPART_CONFIG_ELEMENT, mpce);
String field1 = request.getParameter("field1");
assertNotNull(field1);
@ -2319,7 +2319,7 @@ public class RequestTest
try
{
MultipartConfigElement mpce = new MultipartConfigElement(tmpDir.toString(), -1, -1, 2);
request.setAttribute(Request.__MULTIPART_CONFIG_ELEMENT, mpce);
request.setAttribute(Request.MULTIPART_CONFIG_ELEMENT, mpce);
//We should get an error when we getParams if there was a problem parsing the multipart
request.getPart("xxx");

View File

@ -46,6 +46,7 @@ import org.eclipse.jetty.io.ClientConnectionFactory;
import org.eclipse.jetty.io.ClientConnector;
import org.eclipse.jetty.io.Connection;
import org.eclipse.jetty.io.Content;
import org.eclipse.jetty.io.EndPoint;
import org.eclipse.jetty.io.ssl.SslClientConnectionFactory;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.HttpConfiguration;
@ -140,13 +141,17 @@ public class ClientAuthProxyTest
@Override
public boolean handle(org.eclipse.jetty.server.Request request, Response response, Callback callback)
{
X509Certificate[] certificates = (X509Certificate[])request.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE);
Assertions.assertNotNull(certificates);
X509Certificate certificate = certificates[0];
X500Principal principal = certificate.getSubjectX500Principal();
String body = "%s\r\n%d\r\n".formatted(principal.toString(), org.eclipse.jetty.server.Request.getRemotePort(request));
Content.Sink.write(response, true, body, callback);
return true;
if (request.getAttribute(EndPoint.SslSessionData.ATTRIBUTE) instanceof EndPoint.SslSessionData sslSessionData)
{
X509Certificate[] certificates = sslSessionData.peerCertificates();
Assertions.assertNotNull(certificates);
X509Certificate certificate = certificates[0];
X500Principal principal = certificate.getSubjectX500Principal();
String body = "%s\r\n%d\r\n".formatted(principal.toString(), org.eclipse.jetty.server.Request.getRemotePort(request));
Content.Sink.write(response, true, body, callback);
return true;
}
return false;
}
});
}
@ -213,7 +218,7 @@ public class ClientAuthProxyTest
private static String retrieveUser(HttpServletRequest request)
{
X509Certificate[] certificates = (X509Certificate[])request.getAttribute(SecureRequestCustomizer.PEER_CERTIFICATES_ATTRIBUTE);
X509Certificate[] certificates = (X509Certificate[])request.getAttribute(org.eclipse.jetty.ee9.nested.Request.PEER_CERTIFICATES);
String clientName = certificates[0].getSubjectX500Principal().getName();
Matcher matcher = Pattern.compile("CN=([^,]+)").matcher(clientName);
if (matcher.find())

View File

@ -736,7 +736,7 @@ public class ServletHolder extends Holder<Servlet> implements UserIdentityScope,
{
MultipartConfigElement mpce = ((Registration)_registration).getMultipartConfig();
if (mpce != null)
baseRequest.setAttribute(Request.__MULTIPART_CONFIG_ELEMENT, mpce);
baseRequest.setAttribute(Request.MULTIPART_CONFIG_ELEMENT, mpce);
}
}

View File

@ -38,7 +38,7 @@ import org.eclipse.jetty.util.ssl.SslContextFactory;
public class HTTP1Servlet extends HttpServlet
{
private SslContextFactory sslContextFactory;
private SslContextFactory.Client sslContextFactory;
private HTTP2Client http2Client;
@Override