From 3ec1999f5446ec6dbe59022949737aa33c4cfa84 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Wed, 21 Jun 2023 13:08:25 +0100 Subject: [PATCH 01/56] OF-2559 Implement Netty inbound connection handlers Replace CopyOnWriteMap from MINA; migrating to netty so replacing MINA utility with something similar. Remove MINA-specific stat collector; migrating to netty so removing MINA specific stat collector. For netty we might look to the following in the future to implement a netty-specific stats collector: - https://netty.io/4.0/api/io/netty/handler/ssl/OpenSslSessionStats.html - https://netty.io/4.0/api/io/netty/handler/traffic/package-summary.html --- pom.xml | 1 + xmppserver/pom.xml | 7 + .../mina/management/MINAStatCollector.java | 437 ---------------- .../org/jivesoftware/admin/PluginFilter.java | 4 +- .../openfire/net/ServerStanzaHandler.java | 12 +- .../jivesoftware/openfire/net/SocketUtil.java | 4 +- .../openfire/nio/ConnectionHandler.java | 2 +- .../nio/NettyClientConnectionHandler.java | 72 +++ .../openfire/nio/NettyConnection.java | 493 ++++++++++++++++++ .../openfire/nio/NettyConnectionHandler.java | 208 ++++++++ .../nio/NettyServerConnectionHandler.java | 56 ++ .../openfire/nio/NettyXMPPDecoder.java | 51 ++ .../openfire/nio/XMLLightweightParser.java | 54 +- .../openfire/nio/XMPPDecoder.java | 32 +- .../server/OutgoingSessionPromise.java | 4 +- .../session/LocalIncomingServerSession.java | 114 ++++ .../session/LocalOutgoingServerSession.java | 2 +- .../openfire/spi/ConnectionAcceptor.java | 2 +- .../openfire/spi/ConnectionListener.java | 10 +- .../openfire/spi/ConnectionManagerImpl.java | 2 + .../openfire/spi/MINAConnectionAcceptor.java | 11 - .../openfire/spi/NettyConnectionAcceptor.java | 234 +++++++++ .../nio/XMLLightweightParserTest.java | 10 +- 23 files changed, 1310 insertions(+), 512 deletions(-) delete mode 100644 xmppserver/src/main/java/org/apache/mina/management/MINAStatCollector.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java diff --git a/pom.xml b/pom.xml index cfee390d4d..fced1353ca 100644 --- a/pom.xml +++ b/pom.xml @@ -121,6 +121,7 @@ 9.4.43.v20210629 1.2.5 2.2.1 + 4.1.94.Final 1.70 1.7.36 2.17.1 diff --git a/xmppserver/pom.xml b/xmppserver/pom.xml index b37a510e5e..611f611dfb 100644 --- a/xmppserver/pom.xml +++ b/xmppserver/pom.xml @@ -302,6 +302,13 @@ ${mina.version} + + + io.netty + netty-all + ${netty.version} + + org.bouncycastle diff --git a/xmppserver/src/main/java/org/apache/mina/management/MINAStatCollector.java b/xmppserver/src/main/java/org/apache/mina/management/MINAStatCollector.java deleted file mode 100644 index 8672569754..0000000000 --- a/xmppserver/src/main/java/org/apache/mina/management/MINAStatCollector.java +++ /dev/null @@ -1,437 +0,0 @@ -package org.apache.mina.management; - -import static org.jivesoftware.openfire.spi.ConnectionManagerImpl.EXECUTOR_FILTER_NAME; - -import org.apache.mina.core.service.IoService; -import org.apache.mina.core.service.IoServiceListener; -import org.apache.mina.core.session.IdleStatus; -import org.apache.mina.core.session.IoSession; -import org.apache.mina.filter.executor.ExecutorFilter; -import org.apache.mina.filter.executor.OrderedThreadPoolExecutor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Collects statistics of an {@link IoService}. It's polling all the sessions of a given - * IoService. It's attaching a {@link IoSessionStat} object to all the sessions polled - * and filling the throughput values. - * - * Usage : - *
- * IoService service = ...
- * MINAStatCollector collector = new MINAStatCollector( service );
- * collector.start();
- * 
- * - * By default the {@link org.apache.mina.management.MINAStatCollector} is polling the sessions every 5 seconds. You can - * give a different polling time using a second constructor.

- * - * Note: This class is a spin-off from StatCollector present in - * https://svn.apache.org/repos/asf/mina/branches/1.1/core/src/main/java/org/apache/mina/management. - * - * @author The Apache Directory Project (mina-dev@directory.apache.org) - */ -public class MINAStatCollector { - - private static final Logger Log = LoggerFactory.getLogger(MINAStatCollector.class); - - /** - * The session attribute key for {@link IoSessionStat}. - */ - public static final String KEY = MINAStatCollector.class.getName() + ".stat"; - - - /** - * @noinspection StaticNonFinalField - */ - private static volatile int nextId = 0; - private final int id = nextId ++; - - private final IoService service; - private Worker worker; - private int pollingInterval = 5000; - private Queue polledSessions; - - // resume of session stats, for simplifying acces to the statistics - private AtomicLong totalProcessedSessions = new AtomicLong(); - private AtomicLong totalMsgWritten = new AtomicLong(); - private AtomicLong totalMsgRead = new AtomicLong(); - private AtomicLong totalBytesWritten = new AtomicLong(); - private AtomicLong totalBytesRead = new AtomicLong(); - private AtomicLong totalScheduledWrites = new AtomicLong(); - private AtomicLong totalQueuedEvents = new AtomicLong(); - - private final IoServiceListener serviceListener = new IoServiceListener() - { - - @Override - public void sessionCreated(final IoSession session ) - { - addSession( session ); - } - - @Override - public void sessionDestroyed(final IoSession session ) - { - removeSession( session ); - } - - @Override - public void serviceActivated(final IoService service) { - } - - @Override - public void serviceIdle(final IoService service, final IdleStatus idleStatus) { - } - - @Override - public void serviceDeactivated(final IoService service) { - } - - @Override - public void sessionClosed(final IoSession ioSession) { - } - }; - - /** - * Create a stat collector for the given service with a default polling time of 5 seconds. - * @param service the IoService to inspect - */ - public MINAStatCollector( IoService service ) - { - this( service,5000 ); - } - - /** - * create a stat collector for the given given service - * @param service the IoService to inspect - * @param pollingInterval milliseconds - */ - public MINAStatCollector( IoService service, int pollingInterval ) - { - this.service = service; - this.pollingInterval = pollingInterval; - } - - /** - * Start collecting stats for the {@link IoSession} of the service. - * New sessions or destroyed will be automaticly added or removed. - */ - public void start() - { - synchronized (this) - { - if ( worker != null && worker.isAlive() ) - throw new RuntimeException( "Stat collecting already started" ); - - // add all current sessions - - polledSessions = new ConcurrentLinkedQueue<>(); - - Map sessions = service.getManagedSessions(); - if (sessions != null) { - for (IoSession ioSession : sessions.values()) { - addSession(ioSession); - } - } - - // listen for new ones - service.addListener( serviceListener ); - - // start polling - worker = new Worker(); - worker.start(); - - } - - } - - /** - * Stop collecting stats. all the {@link IoSessionStat} object will be removed of the - * polled session attachements. - */ - public void stop() - { - synchronized (this) - { - service.removeListener( serviceListener ); - - // stop worker - worker.stop = true; - worker.interrupt(); - while( worker.isAlive() ) - { - try - { - worker.join(); - } - catch( InterruptedException e ) - { - //ignore since this is shutdown time - } - } - - for (IoSession session : polledSessions) { - session.removeAttribute(KEY); - } - polledSessions.clear(); - } - } - - /** - * is the stat collector started and polling the {@link IoSession} of the {@link IoService} - * @return true if started - */ - public boolean isRunning() - { - synchronized (this) - { - return worker != null && worker.stop != true; - } - } - - private void addSession( IoSession session ) - { - IoSessionStat sessionStats = new IoSessionStat(); - sessionStats.lastPollingTime = System.currentTimeMillis(); - session.setAttribute( KEY, sessionStats ); - totalProcessedSessions.incrementAndGet(); - polledSessions.add( session ); - } - - private void removeSession( IoSession session ) - { - // remove the session from the list of polled sessions - polledSessions.remove( session ); - - // add the bytes processed between last polling and session closing - // prevent non seen byte with non-connected protocols like HTTP and datagrams - IoSessionStat sessStat = ( IoSessionStat ) session.removeAttribute( KEY ); - - if (sessStat != null) { - totalMsgWritten.addAndGet(session.getWrittenMessages() - sessStat.lastMessageWrite); - totalMsgRead.addAndGet(session.getReadMessages() - sessStat.lastMessageRead); - totalBytesWritten.addAndGet(session.getWrittenBytes() - sessStat.lastByteWrite); - totalBytesRead.addAndGet(session.getReadBytes() - sessStat.lastByteRead); - } - } - - - /** - * total number of sessions processed by the stat collector - * @return number of sessions - */ - public long getTotalProcessedSessions() - { - return totalProcessedSessions.longValue(); - } - - public long getBytesRead() - { - return totalBytesRead.get(); - } - - public long getBytesWritten() - { - return totalBytesWritten.get(); - } - - public long getMsgRead() - { - return totalMsgRead.get(); - } - - public long getMsgWritten() - { - return totalMsgWritten.get(); - } - - public long getScheduledWrites() { - return totalScheduledWrites.get(); - } - - public long getQueuedEvents() { - return totalQueuedEvents.get(); - } - - public long getSessionCount() - { - return polledSessions.size(); - } - - private class Worker extends Thread - { - - boolean stop = false; - - private Worker() - { - super( "StatCollectorWorker-"+id ); - } - - /* - * (non-Javadoc) - * - * @see java.lang.Thread#run() - */ - @Override - public void run() - { - while ( !stop ) - { - // wait polling time - try - { - Thread.sleep( pollingInterval ); - } - catch ( InterruptedException e ) - { - Log.trace("Sleep interrupted"); - } - - long tmpMsgWritten = 0L; - long tmpMsgRead = 0L; - long tmpBytesWritten = 0L; - long tmpBytesRead = 0L; - long tmpScheduledWrites = 0L; - long tmpQueuevedEvents = 0L; - - for (IoSession session : polledSessions) - { - // upadating individual session statistics - IoSessionStat sessStat = ( IoSessionStat ) session.getAttribute( KEY ); - - long currentTimestamp = System.currentTimeMillis(); - // Calculate delta - float pollDelta = (currentTimestamp - sessStat.lastPollingTime) / 1000f; - // Store last polling time of this session - sessStat.lastPollingTime = currentTimestamp; - - long readBytes = session.getReadBytes(); - long writtenBytes = session.getWrittenBytes(); - long readMessages = session.getReadMessages(); - long writtenMessages = session.getWrittenMessages(); - sessStat.byteReadThroughput = (readBytes - sessStat.lastByteRead) / pollDelta; - sessStat.byteWrittenThroughput = (writtenBytes - sessStat.lastByteWrite) / pollDelta; - sessStat.messageReadThroughput = (readMessages - sessStat.lastMessageRead) / pollDelta; - sessStat.messageWrittenThroughput = (writtenMessages - sessStat.lastMessageWrite) / pollDelta; - - tmpMsgWritten += (writtenMessages - sessStat.lastMessageWrite); - tmpMsgRead += (readMessages - sessStat.lastMessageRead); - tmpBytesWritten += (writtenBytes - sessStat.lastByteWrite); - tmpBytesRead += (readBytes - sessStat.lastByteRead); - tmpScheduledWrites += session.getScheduledWriteMessages(); - - ExecutorFilter executorFilter = - (ExecutorFilter) session.getFilterChain().get(EXECUTOR_FILTER_NAME); - if (executorFilter != null) { - Executor executor = executorFilter.getExecutor(); - if (executor instanceof OrderedThreadPoolExecutor) { - tmpQueuevedEvents += ((OrderedThreadPoolExecutor) executor).getActiveCount(); - } - } - - sessStat.lastByteRead = readBytes; - sessStat.lastByteWrite = writtenBytes; - sessStat.lastMessageRead = readMessages; - sessStat.lastMessageWrite = writtenMessages; - - } - - totalMsgWritten.addAndGet(tmpMsgWritten); - totalMsgRead.addAndGet(tmpMsgRead); - totalBytesWritten.addAndGet(tmpBytesWritten); - totalBytesRead.addAndGet(tmpBytesRead); - totalScheduledWrites.set(tmpScheduledWrites); - totalQueuedEvents.set(tmpQueuevedEvents); - } - } - } - - public class IoSessionStat { - long lastByteRead = -1; - - long lastByteWrite = -1; - - long lastMessageRead = -1; - - long lastMessageWrite = -1; - - float byteWrittenThroughput = 0; - - float byteReadThroughput = 0; - - float messageWrittenThroughput = 0; - - float messageReadThroughput = 0; - - // last time the session was polled - long lastPollingTime = System.currentTimeMillis(); - - /** - * Bytes read per second - * @return bytes per second - */ - public float getByteReadThroughput() { - return byteReadThroughput; - } - - /** - * Bytes written per second - * @return bytes per second - */ - public float getByteWrittenThroughput() { - return byteWrittenThroughput; - } - - /** - * Messages read per second - * @return messages per second - */ - public float getMessageReadThroughput() { - return messageReadThroughput; - } - - /** - * Messages written per second - * @return messages per second - */ - public float getMessageWrittenThroughput() { - return messageWrittenThroughput; - } - - /** - * used for the StatCollector, last polling value - */ - long getLastByteRead() { - return lastByteRead; - } - - /** - * used for the StatCollector, last polling value - */ - long getLastByteWrite() { - return lastByteWrite; - } - - /** - * used for the StatCollector, last polling value - */ - long getLastMessageRead() { - return lastMessageRead; - } - - /** - * used for the StatCollector, last polling value - */ - long getLastMessageWrite() { - return lastMessageWrite; - } - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java index 2f4c74e585..d8696773e2 100644 --- a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java +++ b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java @@ -16,7 +16,6 @@ package org.jivesoftware.admin; -import org.apache.mina.util.CopyOnWriteMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** * A servlet filter that plugin classes can use to dynamically register and un-register filter logic. @@ -42,7 +42,7 @@ public class PluginFilter implements Filter { private static final Logger Log = LoggerFactory.getLogger( PluginFilter.class ); - private static final Map> filters = new CopyOnWriteMap<>(); + private static final Map> filters = new ConcurrentHashMap<>(); /** * Adds a filter to the list of filters that will be run on every request of which the URL matches the URL that diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java index 5f4bb015e2..6cff286c39 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java @@ -30,6 +30,8 @@ import org.xmlpull.v1.XmlPullParserException; import org.xmpp.packet.*; +import java.io.IOException; + /** * Handler of XML stanzas sent by remote servers. Remote servers that send stanzas * with no TO or FROM will get their connections closed. Moreover, remote servers @@ -99,9 +101,13 @@ boolean validateJIDs() { void createSession(String serverName, XmlPullParser xpp, Connection connection) throws XmlPullParserException { // The connected client is a server so create an IncomingServerSession - // TODO Finish implementation - //session = LocalIncomingServerSession.createSession(serverName, xpp, connection); - throw new UnsupportedOperationException("Server stanza handler pending implementation"); + // TODO Finish implementation, this is required for netty migration (see ClientStanzaHandler#createSession() which has been implemented) + try { + session = LocalIncomingServerSession.createSession(serverName, xpp, connection, false); + } catch (IOException e) { + Log.error(e.getMessage()); + } +// throw new UnsupportedOperationException("Server stanza handler pending implementation"); } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java index 02abf3ab0a..0abe398a90 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java @@ -69,10 +69,10 @@ public static Map.Entry createSocketToXmppDomain( String xmppDo try { // (re)initialize the socket. - socket = new Socket(); + socket = new Socket(); // AG HERE Log.debug( "Trying to create socket connection to XMPP domain '{}' using remote host: {}:{} (blocks up to {} ms) ...", xmppDomain, realHostname, realPort, socketTimeout ); - socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); + socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); // AG HERE Log.debug( "Successfully created socket connection to XMPP domain '{}' using remote host: {}:{}!", xmppDomain, realHostname, realPort ); return new AbstractMap.SimpleEntry<>(socket, directTLS); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java index 96e841c95a..d89c9330f5 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java @@ -86,7 +86,7 @@ protected ConnectionHandler( ConnectionConfiguration configuration ) { @Override public void sessionOpened(IoSession session) throws Exception { // Create a new XML parser for the new connection. The parser will be used by the XMPPDecoder filter. - final XMLLightweightParser parser = new XMLLightweightParser(StandardCharsets.UTF_8); + final XMLLightweightParser parser = new XMLLightweightParser(); session.setAttribute(XML_PARSER, parser); // Create a new NIOConnection for the new session final NIOConnection connection = createNIOConnection(session); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java new file mode 100644 index 0000000000..1a009a84b2 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java @@ -0,0 +1,72 @@ +package org.jivesoftware.openfire.nio; + +import io.netty.channel.ChannelHandlerContext; +import org.apache.mina.core.session.IdleStatus; +import org.apache.mina.core.session.IoSession; +import org.jivesoftware.openfire.PacketDeliverer; +import org.jivesoftware.openfire.XMPPServer; +import org.jivesoftware.openfire.net.ClientStanzaHandler; +import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.jivesoftware.util.JiveGlobals; +import org.jivesoftware.util.SystemProperty; + +public class NettyClientConnectionHandler extends NettyConnectionHandler{ + + /** + * Enable / disable backup delivery of stanzas to the 'offline message store' of the corresponding user when a stanza + * failed to be delivered on a client connection. When disabled, stanzas that can not be delivered on the connection + * are discarded. + */ + public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) + .setKey("xmpp.client.netty-backup-packet-delivery.enabled") // TODO - rename once MINA-specific is removed and NettyClientConnectionHandler becomes ClientConnectionHandler + .setDefaultValue(true) + .setDynamic(true) + .build(); + + public NettyClientConnectionHandler(ConnectionConfiguration configuration) { + super(configuration); + } + + @Override + NettyConnection createNettyConnection(ChannelHandlerContext ctx) { + final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; + return new NettyConnection(ctx, backupDeliverer, configuration); + } + + @Override + StanzaHandler createStanzaHandler(NettyConnection connection) { + return new ClientStanzaHandler(XMPPServer.getInstance().getPacketRouter(), connection); + + } + + + // TODO Do we need a sessionIdle function for the ClientConnectionHandler specifically? : + /** + * In addition to the functionality provided by the parent class, this + * method will send XMPP ping requests to the remote entity on every first + * invocation of this method (which will occur after a period of half the + * allowed connection idle time has passed, without any IO). + * + * XMPP entities must respond with either an IQ result or an IQ error + * (feature-unavailable) stanza upon receiving the XMPP ping stanza. Both + * responses will be received by Openfire and will cause the connection idle + * count to be reset. + * + * Entities that do not respond to the IQ Ping stanzas can be considered + * dead, and their connection will be closed by the parent class + * implementation on the second invocation of this method. + * + * Note that whitespace pings that are sent by XMPP entities will also cause + * the connection idle count to be reset. + * + * @see ConnectionHandler#sessionIdle(IoSession, IdleStatus) + */ + + @Override + int getMaxIdleTime() + { + return JiveGlobals.getIntProperty( "xmpp.server.idle", 6 * 60 * 1000 ) / 1000; + } + +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java new file mode 100644 index 0000000000..13240b47f1 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -0,0 +1,493 @@ +/* + * Copyright (C) 2005-2008 Jive Software, 2022-2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.jivesoftware.openfire.nio; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.ssl.SslHandler; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.mina.core.buffer.IoBuffer; +import org.jivesoftware.openfire.Connection; +import org.jivesoftware.openfire.ConnectionCloseListener; +import org.jivesoftware.openfire.PacketDeliverer; +import org.jivesoftware.openfire.auth.UnauthorizedException; +import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.net.StartTlsFilter; +import org.jivesoftware.openfire.session.LocalSession; +import org.jivesoftware.openfire.session.Session; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.jivesoftware.openfire.spi.EncryptionArtifactFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xmpp.packet.Packet; +import org.xmpp.packet.StreamError; + +import javax.annotation.Nullable; +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLSession; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.net.UnknownHostException; +import java.nio.charset.CharsetEncoder; +import java.nio.charset.CodingErrorAction; +import java.nio.charset.StandardCharsets; +import java.security.cert.Certificate; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; + +import static org.jivesoftware.openfire.spi.ConnectionManagerImpl.*; + +/** + * Implementation of {@link Connection} interface specific for Netty connections. + * + * @author Matthew Vivian + * @author Alex Gidman + */ +public class NettyConnection implements Connection { + + private static final Logger Log = LoggerFactory.getLogger(NettyConnection.class); + private ConnectionConfiguration configuration; + + /** + * The utf-8 charset for decoding and encoding XMPP packet streams. + */ + public static final String CHARSET = "UTF-8"; + + private LocalSession session; + private ChannelHandlerContext channelHandlerContext; + + final private Map closeListeners = new HashMap<>(); + + /** + * Deliverer to use when the connection is closed or was closed when delivering + * a packet. + */ + private PacketDeliverer backupDeliverer; + private int majorVersion = 1; + private int minorVersion = 0; + private String language = null; + + /** + * TLS policy currently in use for this connection. + */ + private TLSPolicy tlsPolicy = TLSPolicy.optional; + private boolean usingSelfSignedCertificate; + + /** + * Compression policy currently in use for this connection. + */ + private CompressionPolicy compressionPolicy = CompressionPolicy.disabled; + private static final ThreadLocal encoder = new ThreadLocalEncoder(); + + /** + * Flag that specifies if the connection should be considered closed. Closing a NIO connection + * is an asynch operation so instead of waiting for the connection to be actually closed just + * keep this flag to avoid using the connection between #close was used and the socket is actually + * closed. + */ + private AtomicReference state = new AtomicReference<>(State.OPEN); + + /** + * Lock used to ensure the integrity of the underlying IoSession (refer to + * https://issues.apache.org/jira/browse/DIRMINA-653 for details) + *

+ * This lock can be removed once Openfire guarantees a stable delivery + * order, in which case {@link #deliver(Packet)} won't be called + * concurrently any more, which made this lock necessary in the first place. + *

+ */ + private final ReentrantLock ioSessionLock = new ReentrantLock(true); + + public NettyConnection(ChannelHandlerContext channelHandlerContext, @Nullable PacketDeliverer packetDeliverer, ConnectionConfiguration configuration ) { + this.channelHandlerContext = channelHandlerContext; + this.backupDeliverer = packetDeliverer; + this.configuration = configuration; + } + + + @Override + public boolean validate() { + if (isClosed()) { + return false; + } + deliverRawText(" "); + return !isClosed(); + } + + @Override + public void registerCloseListener(ConnectionCloseListener listener, Object callback) { + if (isClosed()) { + listener.onConnectionClose(session); + } + else { + closeListeners.put( listener, callback ); + } + } + + @Override + public void removeCloseListener(ConnectionCloseListener listener) { + closeListeners.remove( listener ); + } + + @Override + public byte[] getAddress() throws UnknownHostException { + final SocketAddress remoteAddress = channelHandlerContext.channel().remoteAddress(); + if (remoteAddress == null) throw new UnknownHostException(); + final InetSocketAddress socketAddress = (InetSocketAddress) remoteAddress; + final InetAddress address = socketAddress.getAddress(); + return address.getAddress(); + } + + @Override + public String getHostAddress() throws UnknownHostException { + final SocketAddress remoteAddress = channelHandlerContext.channel().remoteAddress(); + if (remoteAddress == null) throw new UnknownHostException(); + final InetSocketAddress socketAddress = (InetSocketAddress) remoteAddress; + final InetAddress inetAddress = socketAddress.getAddress(); + return inetAddress.getHostAddress(); + } + + @Override + public String getHostName() throws UnknownHostException { + final SocketAddress remoteAddress = channelHandlerContext.channel().remoteAddress(); + if (remoteAddress == null) throw new UnknownHostException(); + final InetSocketAddress socketAddress = (InetSocketAddress) remoteAddress; + final InetAddress inetAddress = socketAddress.getAddress(); + return inetAddress.getHostName(); + } + + @Override + public Certificate[] getLocalCertificates() { + SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get("ssl"); + + if (sslhandler != null) { + return sslhandler.engine().getSession().getLocalCertificates(); + } + return new Certificate[0]; + } + + @Override + public Certificate[] getPeerCertificates() { + SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get("ssl"); + try { + if (sslhandler != null) { + return sslhandler.engine().getSession().getPeerCertificates(); + } + } catch (SSLPeerUnverifiedException e) { + if (Log.isTraceEnabled()) { + // This is perfectly acceptable when mutual authentication is not enforced by Openfire configuration. + Log.trace("Peer does not offer certificates in session: " + session, e); + } + } + return new Certificate[0]; + } + + @Override + public Optional getTLSProtocolName() { + SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get("ssl"); +// return Optional.ofNullable(sslhandler.engine().getSession().getProtocol()); + return Optional.empty(); + } + + @Override + public Optional getCipherSuiteName() { + SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get("ssl"); +// return Optional.ofNullable(sslhandler.engine().getSession().getCipherSuite()); + return Optional.empty(); + } + + @Override + public void setUsingSelfSignedCertificate(boolean isSelfSigned) { + this.usingSelfSignedCertificate = isSelfSigned; + } + + @Override + public boolean isUsingSelfSignedCertificate() { + return usingSelfSignedCertificate; + } + + @Override + @Nullable + public PacketDeliverer getPacketDeliverer() { + return backupDeliverer; + } + + @Override + public void close() { + close(null); + } + + @Override + public void close(@Nullable final StreamError error) { + if (state.compareAndSet(State.OPEN, State.CLOSED)) { + + // Ensure that the state of this connection, its session and the MINA context are eventually closed. + + if (session != null) { + session.setStatus(Session.Status.CLOSED); + } + + String rawEndStream = ""; + if (error != null) { + rawEndStream = error.toXML(); + } + rawEndStream += ""; + + try { + deliverRawText(rawEndStream); + } catch (Exception e) { + Log.error("Failed to deliver stream close tag: " + e.getMessage()); + } + + try { + // TODO don't block, handle errors async with custom ChannelFutureListener + this.channelHandlerContext.close().addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE).sync(); + } catch (Exception e) { + Log.error("Exception while closing Netty session", e); + } + notifyCloseListeners(); // clean up session, etc. + closeListeners.clear(); + } + } + + @Override + public void systemShutdown() { + close(new StreamError(StreamError.Condition.system_shutdown)); + } + + /** + * Notifies all close listeners that the connection has been closed. + * Used by subclasses to properly finish closing the connection. + */ + private void notifyCloseListeners() { + for( final Map.Entry entry : closeListeners.entrySet() ) + { + if (entry.getKey() != null) { + try { + entry.getKey().onConnectionClose(entry.getValue()); + } catch (Exception e) { + Log.error("Error notifying listener: " + entry.getKey(), e); + } + } + } + } + + @Override + public void init(LocalSession owner) { + session = owner; + } + + @Override + public void reinit(LocalSession owner) { + session = owner; + StanzaHandler stanzaHandler = this.channelHandlerContext.channel().attr(NettyConnectionHandler.HANDLER).get(); + stanzaHandler.setSession(owner); + + // ConnectionCloseListeners are registered with their session instance as a callback object. When re-initializing, + // this object needs to be replaced with the new session instance (or otherwise, the old session will be used + // during the callback. OF-2014 + for ( final Map.Entry entry : closeListeners.entrySet() ) + { + if ( entry.getValue() instanceof LocalSession ) { + entry.setValue( owner ); + } + } + } + + + @Override + public boolean isClosed() { + return state.get() == State.CLOSED; + } + + @Override + @Deprecated // Remove in Openfire 4.9 or later. + public boolean isSecure() { + return isEncrypted(); + } + + @Override + public boolean isEncrypted() { + return false; // TODO with netty + } + + @Override + public void deliver(Packet packet) throws UnauthorizedException { + if (isClosed()) { + if (backupDeliverer != null) { + backupDeliverer.deliver(packet); + } else { + Log.trace("Discarding packet that was due to be delivered on closed connection {}, for which no backup deliverer was configured.", this); + } + } + else { + boolean errorDelivering = false; + try { + ChannelFuture f = channelHandlerContext.writeAndFlush(packet.getElement().asXML()); + // TODO handle errors? + } + catch (Exception e) { + Log.debug("Error delivering packet:\n" + packet, e); + errorDelivering = true; + } + if (errorDelivering) { + close(); + // Retry sending the packet again. Most probably if the packet is a + // Message it will be stored offline + if (backupDeliverer != null) { + backupDeliverer.deliver(packet); + } else { + Log.trace("Discarding packet that failed to be delivered to connection {}, for which no backup deliverer was configured.", this); + } + } + else { + session.incrementServerPacketCount(); + } + } + } + + @Override + public void deliverRawText(String text) { + if (!isClosed()) { + boolean errorDelivering = false; + ChannelFuture f = channelHandlerContext.writeAndFlush(text); + // TODO handle errors? + +// try { + // TODO don't block, handle errors async with custom ChannelFutureListener +// f.addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE); // Removed the sync so this won't throw +// } +// catch (Exception e) { +// Log.error("Error delivering raw text:\n" + text, e); +// e.printStackTrace(); +// errorDelivering = true; +// } + + // Attempt to close the connection if delivering text fails. +// if (errorDelivering) { +// close(); +// } + } + } + + public void startTLS(boolean clientMode, boolean directTLS) throws Exception { + + final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); + // TODO implement ssl filter +// final SslFilter filter; + if ( clientMode ) { +// filter = factory.createClientModeSslFilter(); + } else { +// filter = factory.createServerModeSslFilter(); + } + +// ioSession.getFilterChain().addBefore(EXECUTOR_FILTER_NAME, TLS_FILTER_NAME, filter); + + if (!directTLS) { +// ioSession.getFilterChain().addAfter(TLS_FILTER_NAME, STARTTLS_FILTER_NAME, new StartTlsFilter()); + } + + if ( !clientMode && !directTLS ) { + // Indicate the client that the server is ready to negotiate TLS + deliverRawText( "" ); +// ioSession.getFilterChain().remove(STARTTLS_FILTER_NAME); + } + } + + @Override + public void addCompression() { + System.out.println(this.getClass()+" "+ "addCompression"+" not implemented"); + // TODO with netty + } + + @Override + public void startCompression() { + System.out.println(this.getClass()+" "+ "startCompression"+" not implemented"); + // TODO with netty + } + + @Override + public ConnectionConfiguration getConfiguration() + { + return configuration; + } + + @Override + public int getMajorXMPPVersion() { + return majorVersion; + } + + @Override + public int getMinorXMPPVersion() { + return minorVersion; + } + + @Override + public void setXMPPVersion(int majorVersion, int minorVersion) { + this.majorVersion = majorVersion; + this.minorVersion = minorVersion; + } + + @Override + public boolean isCompressed() { + System.out.println(this.getClass()+" "+ "isCompressed"+" not implemented"); + return false; // TODO with netty + } + + @Override + public CompressionPolicy getCompressionPolicy() { + return compressionPolicy; + } + + @Override + public void setCompressionPolicy(CompressionPolicy compressionPolicy) { + this.compressionPolicy = compressionPolicy; + } + + @Override + public TLSPolicy getTlsPolicy() { + return tlsPolicy; + } + + @Override + public void setTlsPolicy(TLSPolicy tlsPolicy) { + this.tlsPolicy = tlsPolicy; + } + + @Override + public String toString() { + return super.toString() + " Netty Session: " + channelHandlerContext.name(); + } + + private static class ThreadLocalEncoder extends ThreadLocal { + + @Override + protected CharsetEncoder initialValue() { + return StandardCharsets.UTF_8.newEncoder() + .onMalformedInput(CodingErrorAction.REPORT) + .onUnmappableCharacter(CodingErrorAction.REPORT); + } + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java new file mode 100644 index 0000000000..fbaf6e6360 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -0,0 +1,208 @@ +/* + * Copyright (C) 2005-2008 Jive Software, 2022 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.jivesoftware.openfire.nio; + +import io.netty.channel.ChannelHandler.Sharable; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.util.AttributeKey; +import org.apache.mina.core.session.IoSession; +import org.dom4j.io.XMPPPacketReader; +import org.jivesoftware.openfire.Connection; +import org.jivesoftware.openfire.net.MXParser; +import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xmlpull.v1.XmlPullParserException; +import org.xmlpull.v1.XmlPullParserFactory; +import org.xmpp.packet.StreamError; + +import java.nio.charset.StandardCharsets; + +/** + * A NettyConnectionHandler is responsible for creating new sessions, destroying sessions and delivering + * received XML stanzas to the proper StanzaHandler. + * + * @author Matthew Vivian + * @author Alex Gidman + */ +@Sharable +public abstract class NettyConnectionHandler extends SimpleChannelInboundHandler { + + private static final Logger Log = LoggerFactory.getLogger(NettyConnectionHandler.class); + static final AttributeKey XML_PARSER = AttributeKey.valueOf("XML-PARSER"); + static final AttributeKey CONNECTION = AttributeKey.valueOf("CONNECTION"); + static final AttributeKey HANDLER = AttributeKey.valueOf("HANDLER"); + + + protected static final ThreadLocal PARSER_CACHE = new ThreadLocal() + { + @Override + protected XMPPPacketReader initialValue() + { + final XMPPPacketReader parser = new XMPPPacketReader(); + parser.setXPPFactory( factory ); + return parser; + } + }; + /** + * Reuse the same factory for all the connections. + */ + private static XmlPullParserFactory factory = null; + + static { + try { + factory = XmlPullParserFactory.newInstance(MXParser.class.getName(), null); + factory.setNamespaceAware(true); + } + catch (XmlPullParserException e) { + Log.error("Error creating a parser factory", e); + } + } + + /** + * The configuration for new connections. + */ + protected final ConnectionConfiguration configuration; + + protected NettyConnectionHandler(ConnectionConfiguration configuration ) { + this.configuration = configuration; + } + + abstract NettyConnection createNettyConnection(ChannelHandlerContext ctx); + + abstract StanzaHandler createStanzaHandler(NettyConnection connection); + + /** + * Returns the max number of seconds a connection can be idle (both ways) before + * being closed.

+ * + * @return the max number of seconds a connection can be idle. + */ + abstract int getMaxIdleTime(); + + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + System.out.println("Netty XMPP handler added: " + ctx.channel().localAddress()); + // ConnectionHandler.sessionOpened() + + // TODO - do we want a separate parser per-channel? + // Create a new XML parser for the new connection. The parser will be used by the XMPPDecoder filter. + ctx.channel().attr(XML_PARSER).set(new XMLLightweightParser()); + + // Create a new Connection for the new session + final NettyConnection nettyConnection = createNettyConnection(ctx); + ctx.channel().attr(CONNECTION).set(nettyConnection); + + ctx.channel().attr(HANDLER).set(createStanzaHandler(nettyConnection)); + + // Set the max time a connection can be idle before closing it. This amount of seconds + // is divided in two, as Openfire will ping idle clients first (at 50% of the max idle time) + // before disconnecting them (at 100% of the max idle time). This prevents Openfire from + // removing connections without warning. +// TODO idle handler see: https://netty.io/4.0/api/io/netty/handler/timeout/IdleStateHandler.html +// final int idleTime = getMaxIdleTime() / 2; +// if (idleTime > 0) { +// session.getConfig().setIdleTime(IdleStatus.READER_IDLE, idleTime); +// } + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) { + System.out.println("Netty XMPP handler removed: " + ctx.channel().localAddress()); + } + + @Override + public void channelRead0(ChannelHandlerContext ctx, String message) { + // org.jivesoftware.openfire.nio.ConnectionHandler.messageReceived + + // Get the parser to use to process stanza. For optimization there is going + // to be a parser for each running thread. Each Filter will be executed + // by the Executor placed as the first Filter. So we can have a parser associated + // to each Thread + final XMPPPacketReader parser = PARSER_CACHE.get(); + + // Update counter of read bytes + // updateReadBytesCounter(session); TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- + + + //System.out.println("RCVD: " + message); + // Let the stanza handler process the received stanza + try { + ctx.channel().attr(HANDLER).get().process(message, parser); + } catch (Throwable e) { // Make sure to catch Throwable, not (only) Exception! See OF-2367 + Log.error("Closing connection due to error while processing message: {}", message, e); + final Connection connection = ctx.channel().attr(CONNECTION).get(); + if ( connection != null ) { + connection.close(new StreamError(StreamError.Condition.internal_server_error, "An error occurred while processing data raw inbound data.")); + } + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + // Close the connection when an exception is raised. + Log.error(cause.getMessage(), cause); + ctx.close(); + } + + + + + /** + * Updates the system counter of read bytes. This information is used by the incoming + * bytes statistic. + * + * @param ctx the session that read more bytes from the socket. + */ + private void updateReadBytesCounter(ChannelHandlerContext ctx) { +// TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- +// long currentBytes = session.getReadBytes(); +// Long prevBytes = (Long) session.getAttribute("_read_bytes"); +// long delta; +// if (prevBytes == null) { +// delta = currentBytes; +// } +// else { +// delta = currentBytes - prevBytes; +// } +// session.setAttribute("_read_bytes", currentBytes); +// ServerTrafficCounter.incrementIncomingCounter(delta); + } + + /** + * Updates the system counter of written bytes. This information is used by the outgoing + * bytes statistic. + * + * @param session the session that wrote more bytes to the socket. + */ + private void updateWrittenBytesCounter(IoSession session) { +// TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- +// long currentBytes = session.getWrittenBytes(); +// Long prevBytes = (Long) session.getAttribute("_written_bytes"); +// long delta; +// if (prevBytes == null) { +// delta = currentBytes; +// } +// else { +// delta = currentBytes - prevBytes; +// } +// session.setAttribute("_written_bytes", currentBytes); +// ServerTrafficCounter.incrementOutgoingCounter(delta); + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java new file mode 100644 index 0000000000..9435f4fc4c --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -0,0 +1,56 @@ +package org.jivesoftware.openfire.nio; + +import io.netty.channel.ChannelHandlerContext; +import org.jivesoftware.openfire.PacketDeliverer; +import org.jivesoftware.openfire.XMPPServer; +import org.jivesoftware.openfire.net.ServerStanzaHandler; +import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.jivesoftware.util.JiveGlobals; +import org.jivesoftware.util.SystemProperty; + +/** + * ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created and how to build and configure + * a {@link NettyConnection}. + * + * @author Guus der Kinderen, guus.der.kinderen@gmail.com + */ +public class NettyServerConnectionHandler extends NettyConnectionHandler +{ + /** + * Enable / disable backup delivery of stanzas to the XMPP server itself when a stanza failed to be delivered on a + * server-to-server connection. When disabled, stanzas that can not be delivered on the connection are discarded. + */ + public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) + .setKey("xmpp.server.netty-backup-packet-delivery.enabled") // TODO - rename once MINA-specific is removed and NettyServerConnectionHandler becomes ServerConnectionHandler + .setDefaultValue(true) + .setDynamic(true) + .build(); + + public NettyServerConnectionHandler(ConnectionConfiguration configuration) + { + super(configuration); + } + + @Override + NettyConnection createNettyConnection(ChannelHandlerContext ctx) { + final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? XMPPServer.getInstance().getPacketDeliverer() : null; + return new NettyConnection(ctx, backupDeliverer, configuration); + } + + @Override + StanzaHandler createStanzaHandler(NettyConnection connection) { + return new ServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection ); + } + + + + // TBD V + + + + int getMaxIdleTime() + { + return JiveGlobals.getIntProperty( "xmpp.server.idle", 6 * 60 * 1000 ) / 1000; + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java new file mode 100644 index 0000000000..de05a41f9f --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2005-2008 Jive Software. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.jivesoftware.openfire.nio; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.util.CharsetUtil; + +import java.util.Arrays; +import java.util.List; + +/** + * Decoder class that parses ByteBuffers and generates XML stanzas. Generated + * stanzas are then passed to the next filters. + */ +public class NettyXMPPDecoder extends ByteToMessageDecoder { + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { + // Get the XML light parser from the channel + XMLLightweightParser parser = ctx.channel().attr(NettyConnectionHandler.XML_PARSER).get(); + + if (parser.isMaxBufferSizeExceeded()) { + in.release(); + return; + } + + // Parse as many stanzas as possible from the received data + char[] readChars = in.readCharSequence(in.readableBytes(), CharsetUtil.UTF_8).toString().toCharArray(); + parser.read(readChars); + + if (parser.areThereMsgs()) { + out.addAll(Arrays.asList(parser.getMsgs())); + } + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java index dd6dc9213d..f2ef6e0a32 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java @@ -16,20 +16,15 @@ package org.jivesoftware.openfire.nio; -import java.nio.CharBuffer; -import java.nio.charset.Charset; -import java.nio.charset.CharsetDecoder; -import java.nio.charset.CodingErrorAction; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import org.apache.mina.core.buffer.IoBuffer; import org.apache.mina.filter.codec.ProtocolDecoderException; import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.PropertyEventDispatcher; import org.jivesoftware.util.PropertyEventListener; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + /** * This is a Light-Weight XML Parser. * It read data from a channel and collect data until data are available in @@ -93,11 +88,10 @@ class XMLLightweightParser { // List with all finished messages found. protected List msgs = new ArrayList<>(); private int depth = 0; + private boolean maxBufferSizeExceeded = false; protected boolean insideChildrenTag = false; - CharsetDecoder encoder; - static { // Set default max buffer size to 1MB. If limit is reached then close connection maxBufferSize = JiveGlobals.getIntProperty(MAX_PROPERTY_NAME, 1048576); @@ -105,12 +99,6 @@ class XMLLightweightParser { PropertyEventDispatcher.addListener(new PropertyListener()); } - public XMLLightweightParser(Charset charset) { - encoder = charset.newDecoder() - .onMalformedInput(CodingErrorAction.REPLACE) - .onUnmappableCharacter(CodingErrorAction.REPLACE); - } - /* * true if the parser has found some complete xml message. */ @@ -167,33 +155,25 @@ protected void foundMsg(String msg) throws XMLNotWellFormedException { depth = 0; } - /* - * Main reading method - */ - public void read(IoBuffer byteBuffer) throws Exception { - if (buffer == null) { - // exception was thrown before, avoid duplicate exception(s) - // "read" and discard remaining data - byteBuffer.position(byteBuffer.limit()); - return; - } + public boolean isMaxBufferSizeExceeded() { + return maxBufferSizeExceeded; + } + + public void read(char[] buf) throws Exception { invalidateBuffer(); // Check that the buffer is not bigger than 1 Megabyte. For security reasons // we will abort parsing when 1 Mega of queued chars was found. if (buffer.length() > maxBufferSize) { // purge the local buffer / free memory buffer = null; + // set flag to inform higher level network decoders to stop reading more data + maxBufferSizeExceeded = true; // processing the exception takes quite long - final ProtocolDecoderException ex = new ProtocolDecoderException("Stopped parsing never ending stanza"); + final ProtocolDecoderException ex = new ProtocolDecoderException("Stopped parsing never ending stanza"); // TODO throw an openfire decoder exception (not mina specific) ex.setHexdump("(redacted hex dump of never ending stanza)"); throw ex; } - CharBuffer charBuffer = CharBuffer.allocate(byteBuffer.capacity()); - encoder.reset(); - encoder.decode(byteBuffer.buf(), charBuffer, false); - char[] buf = new char[charBuffer.position()]; - charBuffer.flip(); - charBuffer.get(buf); + int readChar = buf.length; // Just return if nothing was read @@ -283,7 +263,7 @@ else if (ch == '!') { // Add message to the list foundMsg(msg); startLastMsg = end; - } + } } else if (ch == '<') { status = XMLLightweightParser.PRETAIL; insideChildrenTag = true; @@ -303,8 +283,8 @@ else if (ch == '!') { cdataOffset = 0; } } else if (cdataOffset == XMLLightweightParser.CDATA_END.length-1 && ch == XMLLightweightParser.CDATA_END[cdataOffset - 1]) { - // if we are looking for the last CDATA_END char, and we instead found an extra ']' - // char, leave cdataOffset as is and proceed to the next char. This could be a case + // if we are looking for the last CDATA_END char, and we instead found an extra ']' + // char, leave cdataOffset as is and proceed to the next char. This could be a case // where the XML character data ends with multiple square braces. For Example ]]]> } else { cdataOffset = 0; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java index e7ed4a56c1..3c7af111d4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java @@ -21,6 +21,11 @@ import org.apache.mina.filter.codec.CumulativeProtocolDecoder; import org.apache.mina.filter.codec.ProtocolDecoderOutput; +import java.nio.CharBuffer; +import java.nio.charset.CharsetDecoder; +import java.nio.charset.CodingErrorAction; +import java.nio.charset.StandardCharsets; + /** * Decoder class that parses ByteBuffers and generates XML stanzas. Generated * stanzas are then passed to the next filters. @@ -30,18 +35,37 @@ public class XMPPDecoder extends CumulativeProtocolDecoder { @Override - protected boolean doDecode(IoSession session, IoBuffer in, ProtocolDecoderOutput out) - throws Exception { + protected boolean doDecode(IoSession session, IoBuffer in, ProtocolDecoderOutput out) throws Exception { // Get the XML light parser from the IoSession XMLLightweightParser parser = (XMLLightweightParser) session.getAttribute(ConnectionHandler.XML_PARSER); + + if (parser.isMaxBufferSizeExceeded()) { + // exception was thrown before, avoid duplicate exception(s) + // "read" and discard remaining data + in.position(in.limit()); + return true; + } + + CharBuffer charBuffer = CharBuffer.allocate(in.capacity()); + + CharsetDecoder encoder = StandardCharsets.UTF_8.newDecoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE); + encoder.reset(); + encoder.decode(in.buf(), charBuffer, false); + + char[] buf = new char[charBuffer.position()]; + charBuffer.flip(); + charBuffer.get(buf); + // Parse as many stanzas as possible from the received data - parser.read(in); + parser.read(buf); if (parser.areThereMsgs()) { for (String stanza : parser.getMsgs()) { out.write(stanza); } } - return !in.hasRemaining(); + return !in.hasRemaining(); // AKA return isEmpty() } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java index 14d0271b80..942b7c45a0 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java @@ -287,7 +287,7 @@ public void run() { Log.trace("Finished processing {}", domainPair); } - private RoutableChannelHandler establishConnection() throws Exception { + private RoutableChannelHandler establishConnection() throws Exception { // TODO: AG - Maybe here? Log.debug("Start establishing a connection for {}", domainPair); // Create a connection to the remote server from the domain where the packet has been sent boolean created; @@ -295,7 +295,7 @@ private RoutableChannelHandler establishConnection() throws Exception { final Lock lock = serversCache.getLock(domainPair); lock.lock(); try { - created = LocalOutgoingServerSession.authenticateDomain(domainPair); + created = LocalOutgoingServerSession.authenticateDomain(domainPair); //TODO: AG MAGIC HERE } finally { lock.unlock(); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java index 05a4b50740..ff26cf0a0b 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java @@ -224,6 +224,120 @@ public static LocalIncomingServerSession createSession(String serverName, XMPPPa } + public static LocalIncomingServerSession createSession(String serverName, XmlPullParser xpp, + Connection connection, boolean directTLS) throws XmlPullParserException, IOException { + + String version = xpp.getAttributeValue("", "version"); + String fromDomain = xpp.getAttributeValue("", "from"); + String toDomain = xpp.getAttributeValue("", "to"); + int[] serverVersion = version != null ? Session.decodeVersion(version) : new int[] {0,0}; + + if (toDomain == null) { + toDomain = serverName; + } + + // Retrieve list of namespaces declared in current element (OF-2556) + connection.setAdditionalNamespaces(XMPPPacketReader.getPrefixedNamespacesOnCurrentElement(xpp)); + + try { + // Get the stream ID for the new session + StreamID streamID = SessionManager.getInstance().nextStreamID(); + // Create a server Session for the remote server + LocalIncomingServerSession session = SessionManager.getInstance().createIncomingServerSession(connection, streamID, fromDomain); + Log.debug("Creating new session with stream ID '{}' for local '{}' to peer '{}'.", streamID, toDomain, fromDomain); + + // Send the stream header + StringBuilder openingStream = new StringBuilder(); + openingStream.append("= 1) { + openingStream.append(" version=\"1.0\">"); + } else { + openingStream.append('>'); + } + + Log.trace("Outbound opening stream: {}", openingStream); + connection.deliverRawText(openingStream.toString()); + + if (serverVersion[0] >= 1) { + // Remote server is XMPP 1.0 compliant so offer TLS and SASL to establish the connection (and server dialback) + + // Indicate the TLS policy to use for this connection + Connection.TLSPolicy tlsPolicy = connection.getTlsPolicy(); + boolean hasCertificates = false; + try { + hasCertificates = XMPPServer.getInstance().getCertificateStoreManager().getIdentityStore( ConnectionType.SOCKET_S2S ).getStore().size() > 0; + } + catch (Exception e) { + Log.error(e.getMessage(), e); + } + if (Connection.TLSPolicy.required == tlsPolicy && !hasCertificates) { + Log.error("Server session rejected. TLS is required but no certificates " + + "were created."); + return null; + } + connection.setTlsPolicy(hasCertificates ? tlsPolicy : Connection.TLSPolicy.disabled); + } + + // Indicate the compression policy to use for this connection + connection.setCompressionPolicy( connection.getConfiguration().getCompressionPolicy() ); + + StringBuilder sb = new StringBuilder(); + + if (serverVersion[0] >= 1) { + Log.trace("Remote server is XMPP 1.0 compliant so offer TLS and SASL to establish the connection (and server dialback)"); + + sb.append(""); + + if (!directTLS && (connection.getTlsPolicy() == Connection.TLSPolicy.required || connection.getTlsPolicy() == Connection.TLSPolicy.optional)) { + sb.append(""); + if (!ServerDialback.isEnabled()) { + Log.debug("Server dialback is disabled so TLS is required"); + sb.append(""); + } + sb.append(""); + } + + // Include available SASL Mechanisms + sb.append(SASLAuthentication.getSASLMechanisms(session)); + + if (ServerDialback.isEnabled()) { + // Also offer server dialback (when TLS is not required). Server dialback may be offered + // after TLS has been negotiated and a self-signed certificate is being used + sb.append(""); + } + + sb.append(""); + } else { + Log.debug("Don't offer stream-features to pre-1.0 servers, as it confuses them. Sending features to Openfire < 3.7.1 confuses it too - OF-443)"); + } + + Log.trace("Outbound feature advertisement: {}", sb); + connection.deliverRawText(sb.toString()); + + Log.trace("Set the domain or subdomain of the local server targeted by the remote server: {}", serverName); + session.setLocalDomain(serverName); + return session; + } + catch (Exception e) { + Log.error("Error establishing connection from remote server: {}", connection, e); + connection.close(new StreamError(StreamError.Condition.internal_server_error)); + return null; + } + } + + + public LocalIncomingServerSession(String serverName, Connection connection, StreamID streamID, String fromDomain) { super(serverName, connection, streamID); this.fromDomain = fromDomain; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index 44d7a2bce0..b043d5384a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -269,7 +269,7 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai try { final SocketAddress socketAddress = socket.getRemoteSocketAddress(); log.debug( "Opening a new connection to {} {}.", socketAddress, directTLS ? "using directTLS" : "that is initially not encrypted" ); - connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); + connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); //TODO: AG NettyConnection? if (directTLS) { try { connection.startTLS( true, true ); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionAcceptor.java index e4c4394b12..f35d1986e7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionAcceptor.java @@ -34,7 +34,7 @@ public ConnectionAcceptor( ConnectionConfiguration configuration ) /** * Makes the instance start accepting connections. * - * An invocation of this method on an instance that is already started should have no effect (to the extend that the + * An invocation of this method on an instance that is already started should have no effect (to the extent that the * instance should continue to accept connections without interruption or configuration changes). */ abstract public void start(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java index 0eab37355b..88f3f20cb3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java @@ -227,12 +227,10 @@ public synchronized void start() } Log.debug( "Starting..." ); - if ( getType() == ConnectionType.SOCKET_S2S ) - { - connectionAcceptor = new LegacyConnectionAcceptor( generateConnectionConfiguration() ); - } - else - { + if ( getType() == ConnectionType.SOCKET_S2S || getType() == ConnectionType.SOCKET_C2S ) { + connectionAcceptor = new NettyConnectionAcceptor( generateConnectionConfiguration() ); +// connectionAcceptor = new LegacyConnectionAcceptor( generateConnectionConfiguration() ); + } else { connectionAcceptor = new MINAConnectionAcceptor( generateConnectionConfiguration() ); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java index 5518d53239..2eac63cbfe 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java @@ -354,6 +354,8 @@ private synchronized void stopListeners() } } + NettyConnectionAcceptor.shutdownEventLoopGroups(); + // Stop the HTTP client listener. try { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java index a1c323ed86..0719d8bede 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java @@ -1,15 +1,9 @@ package org.jivesoftware.openfire.spi; import org.apache.mina.core.filterchain.DefaultIoFilterChainBuilder; -import org.apache.mina.core.service.IoService; -import org.apache.mina.core.service.IoServiceListener; -import org.apache.mina.core.session.IdleStatus; -import org.apache.mina.core.session.IoSession; import org.apache.mina.filter.codec.ProtocolCodecFilter; import org.apache.mina.filter.executor.ExecutorFilter; import org.apache.mina.filter.ssl.SslFilter; -import org.apache.mina.integration.jmx.IoServiceMBean; -import org.apache.mina.integration.jmx.IoSessionMBean; import org.apache.mina.transport.socket.SocketSessionConfig; import org.apache.mina.transport.socket.nio.NioSocketAcceptor; import org.jivesoftware.openfire.Connection; @@ -23,12 +17,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; -import javax.management.JMException; -import javax.management.MBeanServer; -import javax.management.MalformedObjectNameException; import javax.management.ObjectName; -import java.lang.management.ManagementFactory; import java.net.InetSocketAddress; import java.security.KeyManagementException; import java.security.KeyStoreException; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java new file mode 100644 index 0000000000..572b057116 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -0,0 +1,234 @@ +package org.jivesoftware.openfire.spi; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.handler.codec.string.StringDecoder; +import io.netty.handler.codec.string.StringEncoder; +import org.apache.mina.transport.socket.SocketSessionConfig; +import org.apache.mina.transport.socket.nio.NioSocketAcceptor; +import org.jivesoftware.openfire.Connection; +import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; +import org.jivesoftware.openfire.nio.NettyConnectionHandler; +import org.jivesoftware.openfire.nio.NettyServerConnectionHandler; +import org.jivesoftware.openfire.nio.NettyXMPPDecoder; +import org.jivesoftware.util.JiveGlobals; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetSocketAddress; + +/** + * This class is responsible for accepting new (socket) connections, using Java NIO implementation provided by the + * Apache MINA framework. + * + * @author Guus der Kinderen, guus.der.kinderen@gmail.com + */ +class NettyConnectionAcceptor extends ConnectionAcceptor { + // NioEventLoopGroup is a multithreaded event loop that handles I/O operation. + // Netty provides various EventLoopGroup implementations for different kind of transports. + // We are implementing a server-side application in this example, and therefore two + // NioEventLoopGroup will be used. The first one, often called 'boss', accepts an incoming connection. + // The second one, often called 'worker', handles the traffic of the accepted connection once the boss + // accepts the connection and registers the accepted connection to the worker. How many Threads are + // used and how they are mapped to the created Channels depends on the EventLoopGroup implementation + // and may be even configurable via a constructor. + private static final EventLoopGroup BOSS_GROUP = new NioEventLoopGroup(); + private static final EventLoopGroup WORKER_GROUP = new NioEventLoopGroup(); + private final Logger Log; + private final NettyConnectionHandler connectionHandler; + private Channel mainChannel; + + /** + * Instantiates, but not starts, a new instance. + */ + public NettyConnectionAcceptor(ConnectionConfiguration configuration) { + super(configuration); + + String name = configuration.getType().toString().toLowerCase() + (configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode ? "_ssl" : ""); + Log = LoggerFactory.getLogger( NettyConnectionAcceptor.class.getName() + "[" + name + "]" ); + + switch (configuration.getType()) { + case SOCKET_S2S: + connectionHandler = new NettyServerConnectionHandler(configuration); + break; + case SOCKET_C2S: + connectionHandler = new NettyClientConnectionHandler(configuration); + break; + default: + throw new IllegalStateException("This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType()); + } +// connectionHandler = new NettyServerConnectionHandler( configuration ); + +// switch ( configuration.getType() ) +// { +// case SOCKET_S2S: +// connectionHandler = new ServerConnectionHandler( configuration ); +// break; +// case SOCKET_C2S: +// connectionHandler = new ClientConnectionHandler( configuration ); +// break; +// case COMPONENT: +// connectionHandler = new ComponentConnectionHandler( configuration ); +// break; +// case CONNECTION_MANAGER: +// connectionHandler = new MultiplexerConnectionHandler( configuration ); +// break; +// default: +// throw new IllegalStateException( "This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType() ); +// } + } + + private static NioSocketAcceptor buildSocketAcceptor() { + + // TODO consider configuring netty with the settings below (i.e. find the netty way of doing this) + + // Create SocketAcceptor with correct number of processors + final int processorCount = JiveGlobals.getIntProperty("xmpp.processor.count", Runtime.getRuntime().availableProcessors()); + + final NioSocketAcceptor socketAcceptor = new NioSocketAcceptor(processorCount); + + // Set that it will be possible to bind a socket if there is a connection in the timeout state. + socketAcceptor.setReuseAddress(true); + + // Set the listen backlog (queue) length. Default is 50. + socketAcceptor.setBacklog(JiveGlobals.getIntProperty("xmpp.socket.backlog", 50)); + + // Set default (low level) settings for new socket connections + final SocketSessionConfig socketSessionConfig = socketAcceptor.getSessionConfig(); + + //socketSessionConfig.setKeepAlive(); + final int receiveBuffer = JiveGlobals.getIntProperty("xmpp.socket.buffer.receive", -1); + if (receiveBuffer > 0) { + socketSessionConfig.setReceiveBufferSize(receiveBuffer); + } + + final int sendBuffer = JiveGlobals.getIntProperty("xmpp.socket.buffer.send", -1); + if (sendBuffer > 0) { + socketSessionConfig.setSendBufferSize(sendBuffer); + } + + final int linger = JiveGlobals.getIntProperty("xmpp.socket.linger", -1); + if (linger > 0) { + socketSessionConfig.setSoLinger(linger); + } + + socketSessionConfig.setTcpNoDelay(JiveGlobals.getBooleanProperty("xmpp.socket.tcp-nodelay", socketSessionConfig.isTcpNoDelay())); + + return socketAcceptor; + } + + /** + * Starts this acceptor by binding the socket acceptor. When the acceptor is already started, a warning will be + * logged and the method invocation is otherwise ignored. + */ + @Override + public synchronized void start() { + System.out.println("Running Netty on port: " + getPort()); + + try { + // ServerBootstrap is a helper class that sets up a server. You can set up the server using + // a Channel directly. However, please note that this is a tedious process, and you do not + // need to do that in most cases. + ServerBootstrap serverBootstrap = new ServerBootstrap(); + serverBootstrap.group(BOSS_GROUP, WORKER_GROUP) + // Here, we specify to use the NioServerSocketChannel class which is used to + // instantiate a new Channel to accept incoming connections. + .channel(NioServerSocketChannel.class) + // The handler specified here will always be evaluated by a newly accepted Channel. + // The ChannelInitializer is a special handler that is purposed to help a user configure + // a new Channel. It is most likely that you want to configure the ChannelPipeline of the + // new Channel by adding some handlers such as DiscardServerHandler to implement your + // network application. As the application gets complicated, it is likely that you will add + // more handlers to the pipeline and extract this anonymous class into a top-level + // class eventually. + .childHandler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) throws Exception { + ch.pipeline().addLast(new NettyXMPPDecoder()); + ch.pipeline().addLast(new StringEncoder()); + ch.pipeline().addLast(connectionHandler); + } + }) + // You can also set the parameters which are specific to the Channel implementation. + // We are writing a TCP/IP server, so we are allowed to set the socket options such as + // tcpNoDelay and keepAlive. Please refer to the apidocs of ChannelOption and the specific + // ChannelConfig implementations to get an overview about the supported ChannelOptions. + .option(ChannelOption.SO_BACKLOG, 128) + // option() is for the NioServerSocketChannel that accepts incoming connections. + // childOption() is for the Channels accepted by the parent ServerChannel, + // which is NioSocketChannel in this case. + .childOption(ChannelOption.SO_KEEPALIVE, true); + + // Bind to the port and start the server to accept incoming connections. + // You can now call the bind() method as many times as you want (with different bind addresses.) + this.mainChannel = serverBootstrap.bind( + new InetSocketAddress(configuration.getBindAddress(), + configuration.getPort())) + .sync() + .channel(); + + } catch (InterruptedException e) { + System.err.println("Error starting " + configuration.getPort() + ": " + e.getMessage()); + Log.error("Error starting: " + configuration.getPort(), e); + closeMainChannel(); + } + } + + /** + * Stops this acceptor by unbinding the socket acceptor. Does nothing when the instance is not started. + */ + @Override + public synchronized void stop() { + System.out.println("Stop called for port: " + getPort()); + closeMainChannel(); + } + + /** + * Shuts down event loop groups if they are not already shutdown - this will close all channels. + */ + public static void shutdownEventLoopGroups() { + if (!BOSS_GROUP.isShuttingDown()) { + BOSS_GROUP.shutdownGracefully(); + } + if (!WORKER_GROUP.isShuttingDown()) { + WORKER_GROUP.shutdownGracefully(); + } + } + + /** + * Determines if this instance is currently in a state where it is actively serving connections. + * + * @return false when this instance is started and is currently being used to serve connections (otherwise true) + */ + @Override + public synchronized boolean isIdle() { + return mainChannel.isActive(); + } + + @Override + public synchronized void reconfigure(ConnectionConfiguration configuration) { + this.configuration = configuration; + + // TODO reconfigure the netty connection + } + + /** + * Close the main channel (this is not synchronous and does not verify the channel has closed). + */ + public void closeMainChannel() { + if (this.mainChannel != null) { + Log.info("Closing channel " + mainChannel); + mainChannel.close(); + } + } + + public synchronized int getPort() { + return configuration.getPort(); + } +} diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java b/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java index 53e7c762d5..ce09c5baf8 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java @@ -71,7 +71,7 @@ public void testOF2329OpenAndCloseWithSpaceAndNewline() throws Exception final IoBuffer buffer = IoBuffer.allocate(input.length(), false); buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); buffer.flip(); - final XMLLightweightParser parser = new XMLLightweightParser(StandardCharsets.UTF_8); + final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. parser.read(buffer); @@ -97,7 +97,7 @@ public void testOF2329OpenAndCloseWithNewline() throws Exception final IoBuffer buffer = IoBuffer.allocate(input.length(), false); buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); buffer.flip(); - final XMLLightweightParser parser = new XMLLightweightParser(StandardCharsets.UTF_8); + final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. parser.read(buffer); @@ -123,7 +123,7 @@ public void testOF2329SelfTerminatingWithSpace() throws Exception final IoBuffer buffer = IoBuffer.allocate(input.length(), false); buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); buffer.flip(); - final XMLLightweightParser parser = new XMLLightweightParser(StandardCharsets.UTF_8); + final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. parser.read(buffer); @@ -149,7 +149,7 @@ public void testOF2329SelfTerminatingWithSpaceAndNewline() throws Exception final IoBuffer buffer = IoBuffer.allocate(input.length(), false); buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); buffer.flip(); - final XMLLightweightParser parser = new XMLLightweightParser(StandardCharsets.UTF_8); + final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. parser.read(buffer); @@ -175,7 +175,7 @@ public void testOF2329SelfTerminatingWithNewline() throws Exception final IoBuffer buffer = IoBuffer.allocate(input.length(), false); buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); buffer.flip(); - final XMLLightweightParser parser = new XMLLightweightParser(StandardCharsets.UTF_8); + final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. parser.read(buffer); From b0f873145a501165a90c609fe571de311afa3c60 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Thu, 29 Jun 2023 10:50:42 +0100 Subject: [PATCH 02/56] feat: OF-2559 - add compression for inbound and outbound traffic for incoming c2s and s2s connections --- .../openfire/nio/NettyConnection.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 13240b47f1..4b82193457 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -21,6 +21,8 @@ import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.compression.JZlibDecoder; +import io.netty.handler.codec.compression.JZlibEncoder; import io.netty.handler.ssl.SslHandler; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; @@ -57,6 +59,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; +import static com.jcraft.jzlib.JZlib.Z_BEST_COMPRESSION; import static org.jivesoftware.openfire.spi.ConnectionManagerImpl.*; /** @@ -418,14 +421,15 @@ public void startTLS(boolean clientMode, boolean directTLS) throws Exception { @Override public void addCompression() { - System.out.println(this.getClass()+" "+ "addCompression"+" not implemented"); - // TODO with netty + // Inbound traffic only + channelHandlerContext.channel().pipeline().addFirst(new JZlibDecoder()); } @Override public void startCompression() { - System.out.println(this.getClass()+" "+ "startCompression"+" not implemented"); - // TODO with netty + // Outbound traffic only + channelHandlerContext.channel().pipeline().addFirst(new JZlibEncoder(Z_BEST_COMPRESSION)); + // Z_BEST_COMPRESSION is the same level as COMPRESSION_MAX in MINA } @Override @@ -452,8 +456,7 @@ public void setXMPPVersion(int majorVersion, int minorVersion) { @Override public boolean isCompressed() { - System.out.println(this.getClass()+" "+ "isCompressed"+" not implemented"); - return false; // TODO with netty + return channelHandlerContext.channel().pipeline().get(JZlibDecoder.class) != null; } @Override From 2c91f910b316bc18e368d79a197340dba4017934 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Thu, 29 Jun 2023 09:13:59 +0100 Subject: [PATCH 03/56] OF-2559 Fixed tests after refactor XMLLightweightParserTest now works with the newly refactored XMLLightweightParser (which had MINA specifics extracted). --- .../nio/XMLLightweightParserTest.java | 39 +++++++++++++++---- 1 file changed, 32 insertions(+), 7 deletions(-) diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java b/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java index ce09c5baf8..0d588da9d9 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java @@ -16,8 +16,13 @@ package org.jivesoftware.openfire.nio; import org.apache.mina.core.buffer.IoBuffer; + +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.nio.CharBuffer; +import java.nio.charset.CharsetDecoder; +import java.nio.charset.CodingErrorAction; import java.nio.charset.StandardCharsets; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -30,6 +35,27 @@ */ public class XMLLightweightParserTest { + private CharsetDecoder encoder; + private XMLLightweightParser parser; + + @BeforeEach + public void setUp() throws Exception { + parser = new XMLLightweightParser(); + + encoder = StandardCharsets.UTF_8.newDecoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE); + } + + private char[] ioBufferToChars(IoBuffer buffer) { + CharBuffer charBuffer = CharBuffer.allocate(buffer.capacity()); + encoder.decode(buffer.buf(), charBuffer, false); + char[] buf = new char[charBuffer.position()]; + charBuffer.flip(); + charBuffer.get(buf); + return buf; + } + /** * Asserts that a start-tag name can be parsed when it is followed by a space character. * @@ -45,10 +71,9 @@ public void testOF2329OpenAndCloseWithSpace() throws Exception final IoBuffer buffer = IoBuffer.allocate(input.length(), false); buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); buffer.flip(); - final XMLLightweightParser parser = new XMLLightweightParser(StandardCharsets.UTF_8); // Execute system under test. - parser.read(buffer); + parser.read(ioBufferToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -74,7 +99,7 @@ public void testOF2329OpenAndCloseWithSpaceAndNewline() throws Exception final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(buffer); + parser.read(ioBufferToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -100,7 +125,7 @@ public void testOF2329OpenAndCloseWithNewline() throws Exception final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(buffer); + parser.read(ioBufferToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -126,7 +151,7 @@ public void testOF2329SelfTerminatingWithSpace() throws Exception final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(buffer); + parser.read(ioBufferToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -152,7 +177,7 @@ public void testOF2329SelfTerminatingWithSpaceAndNewline() throws Exception final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(buffer); + parser.read(ioBufferToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -178,7 +203,7 @@ public void testOF2329SelfTerminatingWithNewline() throws Exception final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(buffer); + parser.read(ioBufferToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. From 00aa6fa6fd747910076d6f079d384ec2479ff112 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Thu, 29 Jun 2023 12:19:57 +0100 Subject: [PATCH 04/56] OF-2559 Removed last of MINA from XMLLightweightParser MINA will wrap the non-mina exception further up the chain, we were not using the hexdump feature the MINA exception added. --- .../nio/InboundBufferSizeException.java | 40 +++++++++++++++++++ .../openfire/nio/XMLLightweightParser.java | 9 ++--- .../openfire/nio/XMPPDecoder.java | 2 +- 3 files changed, 44 insertions(+), 7 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/InboundBufferSizeException.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/InboundBufferSizeException.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/InboundBufferSizeException.java new file mode 100644 index 0000000000..8b584bdb21 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/InboundBufferSizeException.java @@ -0,0 +1,40 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.jivesoftware.openfire.nio; + +/** + * An Exception indicating that the inbound buffer has exceeded its limit. + */ +public class InboundBufferSizeException extends Exception { + + private static final long serialVersionUID = 1L; + + public InboundBufferSizeException() { + super(); + } + + public InboundBufferSizeException(String message, Throwable cause) { + super(message, cause); + } + + public InboundBufferSizeException(String message) { + super(message); + } + + public InboundBufferSizeException(Throwable cause) { + super(cause); + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java index f2ef6e0a32..208a555fc4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java @@ -16,7 +16,6 @@ package org.jivesoftware.openfire.nio; -import org.apache.mina.filter.codec.ProtocolDecoderException; import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.PropertyEventDispatcher; import org.jivesoftware.util.PropertyEventListener; @@ -164,14 +163,12 @@ public void read(char[] buf) throws Exception { // Check that the buffer is not bigger than 1 Megabyte. For security reasons // we will abort parsing when 1 Mega of queued chars was found. if (buffer.length() > maxBufferSize) { - // purge the local buffer / free memory - buffer = null; // set flag to inform higher level network decoders to stop reading more data maxBufferSizeExceeded = true; + // purge the local buffer / free memory + buffer = null; // processing the exception takes quite long - final ProtocolDecoderException ex = new ProtocolDecoderException("Stopped parsing never ending stanza"); // TODO throw an openfire decoder exception (not mina specific) - ex.setHexdump("(redacted hex dump of never ending stanza)"); - throw ex; + throw new InboundBufferSizeException("Stopped parsing never ending stanza"); } int readChar = buf.length; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java index 3c7af111d4..d02f6a8abe 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java @@ -43,7 +43,7 @@ protected boolean doDecode(IoSession session, IoBuffer in, ProtocolDecoderOutput // exception was thrown before, avoid duplicate exception(s) // "read" and discard remaining data in.position(in.limit()); - return true; + return true; // we are empty } CharBuffer charBuffer = CharBuffer.allocate(in.capacity()); From 16a412e792534e37407b6332e468884477b41fa2 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Thu, 29 Jun 2023 14:23:09 +0100 Subject: [PATCH 05/56] fix: OF-2559 - isIdle now returns true when channel open but not connected --- .../openfire/spi/NettyConnectionAcceptor.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 572b057116..749ae57308 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -25,9 +25,10 @@ /** * This class is responsible for accepting new (socket) connections, using Java NIO implementation provided by the - * Apache MINA framework. + * Netty framework. * - * @author Guus der Kinderen, guus.der.kinderen@gmail.com + * @author Matthew Vivian + * @author Alex Gidman */ class NettyConnectionAcceptor extends ConnectionAcceptor { // NioEventLoopGroup is a multithreaded event loop that handles I/O operation. @@ -202,13 +203,14 @@ public static void shutdownEventLoopGroups() { } /** - * Determines if this instance is currently in a state where it is actively serving connections. + * Determines if this instance is currently in a state where it is actively serving connections or not. + * Channel must be open with no connections if it is idle * * @return false when this instance is started and is currently being used to serve connections (otherwise true) */ @Override public synchronized boolean isIdle() { - return mainChannel.isActive(); + return mainChannel.isOpen() && !mainChannel.isActive(); } @Override From 5af394bded1c3d32ad261140f869189446dd4649 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Fri, 30 Jun 2023 09:27:08 +0100 Subject: [PATCH 06/56] feat: OF-2559 - add channel options that match settings in MinaConnectionAcceptor.buildSocketAcceptor function --- .../openfire/spi/NettyConnectionAcceptor.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 749ae57308..64b56dc8da 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -160,12 +160,40 @@ public void initChannel(SocketChannel ch) throws Exception { // We are writing a TCP/IP server, so we are allowed to set the socket options such as // tcpNoDelay and keepAlive. Please refer to the apidocs of ChannelOption and the specific // ChannelConfig implementations to get an overview about the supported ChannelOptions. + + // Set the listen backlog (queue) length. .option(ChannelOption.SO_BACKLOG, 128) // option() is for the NioServerSocketChannel that accepts incoming connections. // childOption() is for the Channels accepted by the parent ServerChannel, // which is NioSocketChannel in this case. .childOption(ChannelOption.SO_KEEPALIVE, true); + + final int sendBuffer = JiveGlobals.getIntProperty( "xmpp.socket.buffer.send", -1 ); + if ( sendBuffer > 0 ) { + // SO_SNDBUF = Socket Option - Send Buffer Size in Bytes + serverBootstrap.childOption(ChannelOption.SO_SNDBUF, sendBuffer); + } + final int receiveBuffer = JiveGlobals.getIntProperty( "xmpp.socket.buffer.receive", -1 ); + if ( receiveBuffer > 0 ) { + // SO_RCVBUF = Socket Option - Receive Buffer Size in Bytes + serverBootstrap.childOption(ChannelOption.SO_RCVBUF, receiveBuffer); + } + final int linger = JiveGlobals.getIntProperty( "xmpp.socket.linger", -1 ); + if ( linger > 0 ) { + serverBootstrap.childOption(ChannelOption.SO_LINGER, receiveBuffer); + } + + serverBootstrap.childOption(ChannelOption.TCP_NODELAY, JiveGlobals.getBooleanProperty( "xmpp.socket.tcp-nodelay", true)); + + // Set that it will be possible to bind a socket if there is a connection in the timeout state. + serverBootstrap.childOption(ChannelOption.SO_REUSEADDR, true); + + // We do not need to throttle sessions for Netty + if ( configuration.getMaxBufferSize() > 0 ) { + Log.warn( "Throttling by using max buffer size not implemented for Netty; a maximum of 1 message per read is implemented instead."); + } + // Bind to the port and start the server to accept incoming connections. // You can now call the bind() method as many times as you want (with different bind addresses.) this.mainChannel = serverBootstrap.bind( @@ -216,7 +244,6 @@ public synchronized boolean isIdle() { @Override public synchronized void reconfigure(ConnectionConfiguration configuration) { this.configuration = configuration; - // TODO reconfigure the netty connection } From 6d9c6f367fea9864c63ef54adf9d82d27ff05e26 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Mon, 3 Jul 2023 09:21:56 +0100 Subject: [PATCH 07/56] feat: OF-2559 - add WriteTimeoutHandler to handle stalled sessions --- .../openfire/spi/NettyConnectionAcceptor.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 64b56dc8da..8cbdba3ab2 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -8,8 +8,8 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioServerSocketChannel; -import io.netty.handler.codec.string.StringDecoder; import io.netty.handler.codec.string.StringEncoder; +import io.netty.handler.timeout.WriteTimeoutHandler; import org.apache.mina.transport.socket.SocketSessionConfig; import org.apache.mina.transport.socket.nio.NioSocketAcceptor; import org.jivesoftware.openfire.Connection; @@ -18,6 +18,7 @@ import org.jivesoftware.openfire.nio.NettyServerConnectionHandler; import org.jivesoftware.openfire.nio.NettyXMPPDecoder; import org.jivesoftware.util.JiveGlobals; +import org.jivesoftware.util.SystemProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +40,16 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { // accepts the connection and registers the accepted connection to the worker. How many Threads are // used and how they are mapped to the created Channels depends on the EventLoopGroup implementation // and may be even configurable via a constructor. + + /** + * Controls the write timeout time in seconds to handle stalled sessionds and prevent DoS + */ + public static final SystemProperty WRITE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Integer.class) + .setKey("xmpp.socket.write-timeout-seconds") + .setDefaultValue(30) + .setDynamic(true) + .build(); + private static final EventLoopGroup BOSS_GROUP = new NioEventLoopGroup(); private static final EventLoopGroup WORKER_GROUP = new NioEventLoopGroup(); private final Logger Log; @@ -153,6 +164,7 @@ public synchronized void start() { public void initChannel(SocketChannel ch) throws Exception { ch.pipeline().addLast(new NettyXMPPDecoder()); ch.pipeline().addLast(new StringEncoder()); + ch.pipeline().addLast("stalledSessionHandler", new WriteTimeoutHandler(WRITE_TIMEOUT_SECONDS.getValue())); ch.pipeline().addLast(connectionHandler); } }) @@ -181,7 +193,7 @@ public void initChannel(SocketChannel ch) throws Exception { } final int linger = JiveGlobals.getIntProperty( "xmpp.socket.linger", -1 ); if ( linger > 0 ) { - serverBootstrap.childOption(ChannelOption.SO_LINGER, receiveBuffer); + serverBootstrap.childOption(ChannelOption.SO_LINGER, linger); } serverBootstrap.childOption(ChannelOption.TCP_NODELAY, JiveGlobals.getBooleanProperty( "xmpp.socket.tcp-nodelay", true)); @@ -194,6 +206,7 @@ public void initChannel(SocketChannel ch) throws Exception { Log.warn( "Throttling by using max buffer size not implemented for Netty; a maximum of 1 message per read is implemented instead."); } + // Bind to the port and start the server to accept incoming connections. // You can now call the bind() method as many times as you want (with different bind addresses.) this.mainChannel = serverBootstrap.bind( From 78f05a77947316e0adc5cd7298b1393eb6ef413d Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Mon, 3 Jul 2023 10:07:58 +0100 Subject: [PATCH 08/56] fix: OF-2559 - improve write timeout system property description and implementation --- i18n/src/main/resources/openfire_i18n.properties | 1 + .../openfire/spi/NettyConnectionAcceptor.java | 11 +++++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/i18n/src/main/resources/openfire_i18n.properties b/i18n/src/main/resources/openfire_i18n.properties index ec7d867d16..c8fb7cfd1c 100644 --- a/i18n/src/main/resources/openfire_i18n.properties +++ b/i18n/src/main/resources/openfire_i18n.properties @@ -1659,6 +1659,7 @@ system_property.hybridAuthProvider.tertiaryProvider.className=The third class th system_property.admin.authorizedJIDs=The bare JID of every admin user for the DefaultAdminProvider system_property.xmpp.auth.ssl.context_protocol=The TLS protocol to use for encryption context initialization, overriding the Java default. system_property.xmpp.socket.ssl.active=Set to true to enable legacy encrypted connections for clients, otherwise false +system_property.xmpp.socket.write-timeout-seconds=The write timeout time in seconds to handle stalled sessions and prevent DoS system_property.xmpp.component.ssl.active=Set to true to enable legacy encrypted connections for external components, otherwise false system_property.xmpp.server.startup.retry.delay=Set to a positive value to allow a retry of a failed startup after the specified duration. system_property.sasl.realm=The realm used for SASL authentication, which can be used when realms that are passed through SASL are different from the XMPP domain name. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 8cbdba3ab2..5d2eca0b7c 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -23,6 +23,8 @@ import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; +import java.time.Duration; +import java.time.temporal.ChronoUnit; /** * This class is responsible for accepting new (socket) connections, using Java NIO implementation provided by the @@ -42,11 +44,12 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { // and may be even configurable via a constructor. /** - * Controls the write timeout time in seconds to handle stalled sessionds and prevent DoS + * Controls the write timeout time in seconds to handle stalled sessions and prevent DoS */ - public static final SystemProperty WRITE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Integer.class) + public static final SystemProperty WRITE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Duration.class) .setKey("xmpp.socket.write-timeout-seconds") - .setDefaultValue(30) + .setDefaultValue(Duration.ofSeconds(30)) + .setChronoUnit(ChronoUnit.SECONDS) .setDynamic(true) .build(); @@ -164,7 +167,7 @@ public synchronized void start() { public void initChannel(SocketChannel ch) throws Exception { ch.pipeline().addLast(new NettyXMPPDecoder()); ch.pipeline().addLast(new StringEncoder()); - ch.pipeline().addLast("stalledSessionHandler", new WriteTimeoutHandler(WRITE_TIMEOUT_SECONDS.getValue())); + ch.pipeline().addLast("stalledSessionHandler", new WriteTimeoutHandler((int)WRITE_TIMEOUT_SECONDS.getValue().getSeconds())); ch.pipeline().addLast(connectionHandler); } }) From 1650fe5003edfcf70de4df91a41763f527faa675 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 3 Jul 2023 11:59:36 +0100 Subject: [PATCH 09/56] OF-2559 Added TLS handler for inbound netty connections Tested with non-netty outbound server to a netty-based inbound, so using TLS 1.2 as restricted by outbound capabilities until we pull netty through into outbound connection. --- .../openfire/net/ServerStanzaHandler.java | 19 ++- .../openfire/net/StanzaHandler.java | 8 +- .../openfire/nio/NettyConnection.java | 37 ++--- .../openfire/nio/NettyConnectionHandler.java | 2 - .../openfire/session/ConnectionSettings.java | 6 + .../spi/EncryptionArtifactFactory.java | 27 ++++ .../openfire/spi/NettyConnectionAcceptor.java | 148 ++++-------------- .../openfire/spi/NettyServerInitializer.java | 44 ++++++ 8 files changed, 147 insertions(+), 144 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java index 6cff286c39..dc9bf55d10 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2005-2008 Jive Software, 2022 Ignite Realtime Foundation. All rights reserved. + * Copyright (C) 2005-2008 Jive Software, 2022-2023 Ignite Realtime Foundation. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ package org.jivesoftware.openfire.net; import org.dom4j.Element; +import org.dom4j.Namespace; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.PacketRouter; import org.jivesoftware.openfire.auth.UnauthorizedException; @@ -31,6 +32,7 @@ import org.xmpp.packet.*; import java.io.IOException; +import java.util.Set; /** * Handler of XML stanzas sent by remote servers. Remote servers that send stanzas @@ -82,6 +84,17 @@ else if ("db".equals(doc.getNamespacePrefix()) && "verify".equals(doc.getName()) return false; } + @Override + protected String getAdditionalNamespaces() { + final Set namespaces = connection.getAdditionalNamespaces(); + if (namespaces.isEmpty()) { + return ""; + } + final StringBuilder sb = new StringBuilder(); + namespaces.forEach(namespace -> sb.append(" ").append(namespace.asXML())); + return sb.toString(); + } + @Override String getNamespace() { return "jabber:server"; @@ -101,13 +114,11 @@ boolean validateJIDs() { void createSession(String serverName, XmlPullParser xpp, Connection connection) throws XmlPullParserException { // The connected client is a server so create an IncomingServerSession - // TODO Finish implementation, this is required for netty migration (see ClientStanzaHandler#createSession() which has been implemented) try { session = LocalIncomingServerSession.createSession(serverName, xpp, connection, false); } catch (IOException e) { - Log.error(e.getMessage()); + Log.error(e.getMessage(), e); } -// throw new UnsupportedOperationException("Server stanza handler pending implementation"); } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java index 47d3559073..3ac232c2b6 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java @@ -619,7 +619,9 @@ protected String getStreamHeader() { sb.append("'?>"); sb.append(" getTLSProtocolName() { - SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get("ssl"); -// return Optional.ofNullable(sslhandler.engine().getSession().getProtocol()); - return Optional.empty(); + SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get(SSL_HANDLER_NAME); + return Optional.ofNullable(sslhandler.engine().getSession().getProtocol()); } @Override public Optional getCipherSuiteName() { - SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get("ssl"); -// return Optional.ofNullable(sslhandler.engine().getSession().getCipherSuite()); - return Optional.empty(); + SslHandler sslhandler = (SslHandler) channelHandlerContext.channel().pipeline().get(SSL_HANDLER_NAME); + return Optional.ofNullable(sslhandler.engine().getSession().getCipherSuite()); } @Override @@ -373,6 +365,7 @@ public void deliver(Packet packet) throws UnauthorizedException { @Override public void deliverRawText(String text) { + //System.out.println("Sending: " + text); if (!isClosed()) { boolean errorDelivering = false; ChannelFuture f = channelHandlerContext.writeAndFlush(text); @@ -398,19 +391,17 @@ public void deliverRawText(String text) { public void startTLS(boolean clientMode, boolean directTLS) throws Exception { final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); - // TODO implement ssl filter -// final SslFilter filter; + SslContext sslContext = factory.createSslContext(); + + final SslHandler sslHandler = sslContext.newHandler(channelHandlerContext.alloc()); + if ( clientMode ) { // filter = factory.createClientModeSslFilter(); } else { // filter = factory.createServerModeSslFilter(); } -// ioSession.getFilterChain().addBefore(EXECUTOR_FILTER_NAME, TLS_FILTER_NAME, filter); - - if (!directTLS) { -// ioSession.getFilterChain().addAfter(TLS_FILTER_NAME, STARTTLS_FILTER_NAME, new StartTlsFilter()); - } + channelHandlerContext.pipeline().addFirst(SSL_HANDLER_NAME, sslHandler); if ( !clientMode && !directTLS ) { // Indicate the client that the server is ready to negotiate TLS diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index fbaf6e6360..a063cdf01a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -129,8 +129,6 @@ public void handlerRemoved(ChannelHandlerContext ctx) { @Override public void channelRead0(ChannelHandlerContext ctx, String message) { - // org.jivesoftware.openfire.nio.ConnectionHandler.messageReceived - // Get the parser to use to process stanza. For optimization there is going // to be a parser for each running thread. Each Filter will be executed // by the Executor placed as the first Filter. So we can have a parser associated diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java index 03a055236a..d731a8a856 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java @@ -59,6 +59,12 @@ public static final class Client { public static final String LOGIN_ANONYM_ALLOWED = "xmpp.client.login.allowedAnonym"; public static final String MAX_THREADS = "xmpp.client.processing.threads"; + + /** + * Used to configure throttling at the network level + * + * @deprecated not implemented for Netty as this uses max of 1 message per read instead + */ public static final String MAX_READ_BUFFER = "xmpp.client.maxReadBufferSize"; public static final String MAX_THREADS_SSL = "xmpp.client_ssl.processing.threads"; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index c3a89701bf..4ac623cceb 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -1,5 +1,8 @@ package org.jivesoftware.openfire.spi; +import io.netty.handler.ssl.ClientAuth; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; import org.apache.mina.filter.ssl.SslFilter; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.jivesoftware.openfire.keystore.OpenfireX509TrustManager; @@ -324,6 +327,30 @@ public synchronized SslContextFactory getSslContextFactory() } } + public SslContext createSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { + getKeyManagers(); + SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); + + // Set policy for checking client certificates. + switch ( configuration.getClientAuth() ) + { + case disabled: + builder.clientAuth(ClientAuth.NONE); + break; + case wanted: + builder.clientAuth(ClientAuth.OPTIONAL); + break; + case needed: + builder.clientAuth(ClientAuth.REQUIRE); + break; + } + + builder.protocols(configuration.getEncryptionProtocols()); + builder.startTls(true); + + return builder.build(); + } + /** * Creates an Apache MINA SslFilter that is configured to use server mode when handshaking. * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 5d2eca0b7c..960c7e6e4e 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -2,29 +2,19 @@ import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.Channel; -import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioServerSocketChannel; -import io.netty.handler.codec.string.StringEncoder; -import io.netty.handler.timeout.WriteTimeoutHandler; -import org.apache.mina.transport.socket.SocketSessionConfig; -import org.apache.mina.transport.socket.nio.NioSocketAcceptor; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; import org.jivesoftware.openfire.nio.NettyConnectionHandler; import org.jivesoftware.openfire.nio.NettyServerConnectionHandler; -import org.jivesoftware.openfire.nio.NettyXMPPDecoder; import org.jivesoftware.util.JiveGlobals; -import org.jivesoftware.util.SystemProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; -import java.time.Duration; -import java.time.temporal.ChronoUnit; /** * This class is responsible for accepting new (socket) connections, using Java NIO implementation provided by the @@ -35,25 +25,25 @@ */ class NettyConnectionAcceptor extends ConnectionAcceptor { // NioEventLoopGroup is a multithreaded event loop that handles I/O operation. - // Netty provides various EventLoopGroup implementations for different kind of transports. - // We are implementing a server-side application in this example, and therefore two - // NioEventLoopGroup will be used. The first one, often called 'boss', accepts an incoming connection. + // The first one, often called 'boss', accepts an incoming connection. // The second one, often called 'worker', handles the traffic of the accepted connection once the boss // accepts the connection and registers the accepted connection to the worker. How many Threads are // used and how they are mapped to the created Channels depends on the EventLoopGroup implementation // and may be even configurable via a constructor. /** - * Controls the write timeout time in seconds to handle stalled sessions and prevent DoS + * A multithreaded event loop that handles I/O operation + *

+ * The 'boss' accepts an incoming connection. */ - public static final SystemProperty WRITE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Duration.class) - .setKey("xmpp.socket.write-timeout-seconds") - .setDefaultValue(Duration.ofSeconds(30)) - .setChronoUnit(ChronoUnit.SECONDS) - .setDynamic(true) - .build(); - private static final EventLoopGroup BOSS_GROUP = new NioEventLoopGroup(); + + /** + * A multithreaded event loop that handles I/O operation + *

+ * The 'worker', handles the traffic of the accepted connection once the boss accepts the connection + * and registers the accepted connection to the worker. + */ private static final EventLoopGroup WORKER_GROUP = new NioEventLoopGroup(); private final Logger Log; private final NettyConnectionHandler connectionHandler; @@ -99,45 +89,6 @@ public NettyConnectionAcceptor(ConnectionConfiguration configuration) { // } } - private static NioSocketAcceptor buildSocketAcceptor() { - - // TODO consider configuring netty with the settings below (i.e. find the netty way of doing this) - - // Create SocketAcceptor with correct number of processors - final int processorCount = JiveGlobals.getIntProperty("xmpp.processor.count", Runtime.getRuntime().availableProcessors()); - - final NioSocketAcceptor socketAcceptor = new NioSocketAcceptor(processorCount); - - // Set that it will be possible to bind a socket if there is a connection in the timeout state. - socketAcceptor.setReuseAddress(true); - - // Set the listen backlog (queue) length. Default is 50. - socketAcceptor.setBacklog(JiveGlobals.getIntProperty("xmpp.socket.backlog", 50)); - - // Set default (low level) settings for new socket connections - final SocketSessionConfig socketSessionConfig = socketAcceptor.getSessionConfig(); - - //socketSessionConfig.setKeepAlive(); - final int receiveBuffer = JiveGlobals.getIntProperty("xmpp.socket.buffer.receive", -1); - if (receiveBuffer > 0) { - socketSessionConfig.setReceiveBufferSize(receiveBuffer); - } - - final int sendBuffer = JiveGlobals.getIntProperty("xmpp.socket.buffer.send", -1); - if (sendBuffer > 0) { - socketSessionConfig.setSendBufferSize(sendBuffer); - } - - final int linger = JiveGlobals.getIntProperty("xmpp.socket.linger", -1); - if (linger > 0) { - socketSessionConfig.setSoLinger(linger); - } - - socketSessionConfig.setTcpNoDelay(JiveGlobals.getBooleanProperty("xmpp.socket.tcp-nodelay", socketSessionConfig.isTcpNoDelay())); - - return socketAcceptor; - } - /** * Starts this acceptor by binding the socket acceptor. When the acceptor is already started, a warning will be * logged and the method invocation is otherwise ignored. @@ -147,42 +98,23 @@ public synchronized void start() { System.out.println("Running Netty on port: " + getPort()); try { - // ServerBootstrap is a helper class that sets up a server. You can set up the server using - // a Channel directly. However, please note that this is a tedious process, and you do not - // need to do that in most cases. + // ServerBootstrap is a helper class that sets up a server ServerBootstrap serverBootstrap = new ServerBootstrap(); serverBootstrap.group(BOSS_GROUP, WORKER_GROUP) - // Here, we specify to use the NioServerSocketChannel class which is used to - // instantiate a new Channel to accept incoming connections. + // Instantiate a new Channel to accept incoming connections. .channel(NioServerSocketChannel.class) // The handler specified here will always be evaluated by a newly accepted Channel. - // The ChannelInitializer is a special handler that is purposed to help a user configure - // a new Channel. It is most likely that you want to configure the ChannelPipeline of the - // new Channel by adding some handlers such as DiscardServerHandler to implement your - // network application. As the application gets complicated, it is likely that you will add - // more handlers to the pipeline and extract this anonymous class into a top-level - // class eventually. - .childHandler(new ChannelInitializer() { - @Override - public void initChannel(SocketChannel ch) throws Exception { - ch.pipeline().addLast(new NettyXMPPDecoder()); - ch.pipeline().addLast(new StringEncoder()); - ch.pipeline().addLast("stalledSessionHandler", new WriteTimeoutHandler((int)WRITE_TIMEOUT_SECONDS.getValue().getSeconds())); - ch.pipeline().addLast(connectionHandler); - } - }) - // You can also set the parameters which are specific to the Channel implementation. - // We are writing a TCP/IP server, so we are allowed to set the socket options such as - // tcpNoDelay and keepAlive. Please refer to the apidocs of ChannelOption and the specific - // ChannelConfig implementations to get an overview about the supported ChannelOptions. - + .childHandler(new NettyServerInitializer(connectionHandler)) // Set the listen backlog (queue) length. - .option(ChannelOption.SO_BACKLOG, 128) + .option(ChannelOption.SO_BACKLOG, JiveGlobals.getIntProperty("xmpp.socket.backlog", 50)) // option() is for the NioServerSocketChannel that accepts incoming connections. // childOption() is for the Channels accepted by the parent ServerChannel, // which is NioSocketChannel in this case. - .childOption(ChannelOption.SO_KEEPALIVE, true); - + .childOption(ChannelOption.SO_KEEPALIVE, true) + // Setting TCP_NODELAY to false enables the Nagle algorithm, which delays sending small successive packets + .childOption(ChannelOption.TCP_NODELAY, JiveGlobals.getBooleanProperty( "xmpp.socket.tcp-nodelay", true)) + // Set that it will be possible to bind a socket if there is a connection in the timeout state. + .childOption(ChannelOption.SO_REUSEADDR, true); final int sendBuffer = JiveGlobals.getIntProperty( "xmpp.socket.buffer.send", -1 ); if ( sendBuffer > 0 ) { @@ -199,27 +131,16 @@ public void initChannel(SocketChannel ch) throws Exception { serverBootstrap.childOption(ChannelOption.SO_LINGER, linger); } - serverBootstrap.childOption(ChannelOption.TCP_NODELAY, JiveGlobals.getBooleanProperty( "xmpp.socket.tcp-nodelay", true)); - - // Set that it will be possible to bind a socket if there is a connection in the timeout state. - serverBootstrap.childOption(ChannelOption.SO_REUSEADDR, true); - - // We do not need to throttle sessions for Netty - if ( configuration.getMaxBufferSize() > 0 ) { - Log.warn( "Throttling by using max buffer size not implemented for Netty; a maximum of 1 message per read is implemented instead."); - } - - // Bind to the port and start the server to accept incoming connections. - // You can now call the bind() method as many times as you want (with different bind addresses.) this.mainChannel = serverBootstrap.bind( - new InetSocketAddress(configuration.getBindAddress(), - configuration.getPort())) + new InetSocketAddress( + configuration.getBindAddress(), + configuration.getPort()) + ) .sync() .channel(); } catch (InterruptedException e) { - System.err.println("Error starting " + configuration.getPort() + ": " + e.getMessage()); Log.error("Error starting: " + configuration.getPort(), e); closeMainChannel(); } @@ -230,10 +151,19 @@ public void initChannel(SocketChannel ch) throws Exception { */ @Override public synchronized void stop() { - System.out.println("Stop called for port: " + getPort()); closeMainChannel(); } + /** + * Close the main channel (this is not synchronous and does not verify the channel has closed). + */ + private void closeMainChannel() { + if (this.mainChannel != null) { + Log.info("Closing channel " + mainChannel); + mainChannel.close(); + } + } + /** * Shuts down event loop groups if they are not already shutdown - this will close all channels. */ @@ -263,16 +193,6 @@ public synchronized void reconfigure(ConnectionConfiguration configuration) { // TODO reconfigure the netty connection } - /** - * Close the main channel (this is not synchronous and does not verify the channel has closed). - */ - public void closeMainChannel() { - if (this.mainChannel != null) { - Log.info("Closing channel " + mainChannel); - mainChannel.close(); - } - } - public synchronized int getPort() { return configuration.getPort(); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java new file mode 100644 index 0000000000..773496ddb4 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java @@ -0,0 +1,44 @@ +package org.jivesoftware.openfire.spi; + +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.string.StringEncoder; +import io.netty.handler.timeout.WriteTimeoutHandler; +import org.jivesoftware.openfire.nio.NettyConnectionHandler; +import org.jivesoftware.openfire.nio.NettyXMPPDecoder; +import org.jivesoftware.util.SystemProperty; + +import java.time.Duration; +import java.time.temporal.ChronoUnit; + +/** + * Creates a newly configured {@link ChannelPipeline} for a new channel. + */ +public class NettyServerInitializer extends ChannelInitializer { + + /** + * Controls the write timeout time in seconds to handle stalled sessions and prevent DoS + */ + public static final SystemProperty WRITE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Duration.class) + .setKey("xmpp.socket.write-timeout-seconds") + .setDefaultValue(Duration.ofSeconds(30)) + .setChronoUnit(ChronoUnit.SECONDS) + .setDynamic(true) + .build(); + + private final NettyConnectionHandler businessLogicHandler; + + public NettyServerInitializer(NettyConnectionHandler businessLogicHandler) { + this.businessLogicHandler = businessLogicHandler; + } + + @Override + public void initChannel(SocketChannel ch) { + ch.pipeline().addLast(new NettyXMPPDecoder()); + ch.pipeline().addLast(new StringEncoder()); + ch.pipeline().addLast("stalledSessionHandler", new WriteTimeoutHandler(Math.toIntExact(WRITE_TIMEOUT_SECONDS.getValue().getSeconds()))); + ch.pipeline().addLast(businessLogicHandler); + + } +} From 48c9599779c241ba478e775ef8bf814aea8616c0 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Wed, 5 Jul 2023 14:59:36 +0100 Subject: [PATCH 10/56] OF-2559 S2S outbound with Netty TLS 1.2 & 1.3 working with S2S --- .../net/RespondingServerStanzaHandler.java | 303 ++++++++++++++++++ .../openfire/net/SocketConnection.java | 1 + .../openfire/net/StanzaHandler.java | 6 +- .../openfire/nio/NettyConnection.java | 16 +- .../openfire/nio/NettyConnectionHandler.java | 2 +- .../nio/NettyOutboundConnectionHandler.java | 41 +++ .../nio/NettyServerConnectionHandler.java | 7 +- .../openfire/nio/NettySessionInitializer.java | 145 +++++++++ .../server/OutgoingSessionPromise.java | 33 +- .../openfire/server/ServerDialback.java | 33 +- .../session/LocalOutgoingServerSession.java | 187 +---------- .../spi/EncryptionArtifactFactory.java | 20 +- .../openfire/spi/RoutingTableImpl.java | 4 +- 13 files changed, 582 insertions(+), 216 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java new file mode 100644 index 0000000000..680d6a88ba --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -0,0 +1,303 @@ +package org.jivesoftware.openfire.net; + +import org.dom4j.DocumentException; +import org.dom4j.DocumentHelper; +import org.dom4j.Element; +import org.dom4j.Namespace; +import org.dom4j.io.XMPPPacketReader; +import org.jivesoftware.openfire.Connection; +import org.jivesoftware.openfire.PacketRouter; +import org.jivesoftware.openfire.auth.UnauthorizedException; +import org.jivesoftware.openfire.server.ServerDialback; +import org.jivesoftware.openfire.session.*; +import org.jivesoftware.openfire.spi.BasicStreamIDFactory; +import org.jivesoftware.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xmlpull.v1.XmlPullParser; +import org.xmlpull.v1.XmlPullParserException; +import org.xmpp.packet.JID; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +public class RespondingServerStanzaHandler extends StanzaHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RespondingServerStanzaHandler.class); + private final DomainPair domainPair; + + private boolean isSessionAuthenticated = false; + + /** + * Creates a dedicated reader for a socket. + * + * @param router the router for sending packets that were read. + * @param connection the connection being read. + * @param domainPair the local and remote domains + */ + public RespondingServerStanzaHandler(PacketRouter router, Connection connection, DomainPair domainPair) { + super(router, connection); + this.domainPair = domainPair; + } + + private static boolean remoteFeaturesContainsStartTLS(Element doc) { + return doc.element("starttls") != null; + } + + private static boolean isSaslExternalOfferred(Element doc) { + boolean saslEXTERNALoffered = false; + if (doc.element("mechanisms") != null) { + Iterator it = doc.element("mechanisms").elementIterator(); + while (it.hasNext()) { + Element mechanism = it.next(); + if ("EXTERNAL".equals(mechanism.getTextTrim())) { + saslEXTERNALoffered = true; + break; + } + } + } + return saslEXTERNALoffered; + } + + private static boolean isDialbackOffered(Element doc) { + return doc.element("dialback") != null; + } + + @Override + protected boolean isStartOfStream(String xml) { + final boolean isStartOfStream = super.isStartOfStream(xml); + if (isStartOfStream) { + // We initiate the stream for a RespondingServerStanzaHandler, so we need to add the stream namespace + // Pull namespaces off of the stream:stream stanza and add them to the additional + List receivedNamespaces = null; + try { + receivedNamespaces = DocumentHelper.parseText(xml + "").getRootElement().declaredNamespaces(); + Set additionalNamespaces = receivedNamespaces + .stream() + .filter(RespondingServerStanzaHandler::isRelevantNamespace) + .collect(Collectors.toSet()); + connection.setAdditionalNamespaces(additionalNamespaces); + } catch (DocumentException e) { + LOG.error("Failed extract additional namespaces", e); + } + } + + return isStartOfStream; + } + + @Override + protected void initiateSession(String stanza, XMPPPacketReader reader) throws Exception { + if (!sessionCreated) { + super.initiateSession(stanza, reader); + } + } + + private static boolean isRelevantNamespace(Namespace ns) { + return !XMPPPacketReader.IGNORED_NAMESPACE_ON_STANZA.contains(ns.getURI()); + } + + @Override + boolean processUnknowPacket(Element doc) throws UnauthorizedException { + String rootTagName = doc.getName(); + + // Handle features + if ("features".equals(rootTagName)) { + LOG.debug("Check if both us as well as the remote server have enabled STARTTLS and/or dialback ..."); + + // Encryption ------ + if (shouldUseTls() && remoteFeaturesContainsStartTLS(doc)) { + LOG.debug("Both us and the remote server support the STARTTLS feature. Encrypt and authenticate the connection with TLS & SASL..."); + LOG.debug("Indicating we want TLS and wait for response."); + connection.deliverRawText(""); + startedTLS = true; + return true; + } else if (mustUseTls() && !connection.isEncrypted()) { + LOG.debug("I MUST use TLS but I have no StartTLS in features."); + return false; + } + + // Authentication ------ + final boolean saslExternalOffered = isSaslExternalOfferred(doc); + final boolean dialbackOffered = isDialbackOffered(doc); + LOG.debug("Remote server is offering dialback: {}, EXTERNAL SASL: {}", dialbackOffered, saslExternalOffered); + + // First, try SASL + if (saslExternalOffered) { + LOG.debug("Trying to authenticate with EXTERNAL SASL."); + LOG.debug("Starting EXTERNAL SASL for: " + domainPair); + + StringBuilder sb = new StringBuilder(); + sb.append(""); + sb.append(StringUtils.encodeBase64(domainPair.getLocal())); + sb.append(""); + connection.deliverRawText(sb.toString()); + startedSASL = true; + return true; + } else if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + // Next, try dialback + LOG.debug("Trying to authenticate using dialback."); + LOG.debug("[Acting as Originating Server: Authenticate domain: " + domainPair.getLocal() + " with a RS in the domain of: " + domainPair.getRemote() + " (id: " + session.getStreamID() + ")]"); + ServerDialback dialback = new ServerDialback(connection, domainPair); + dialback.createAndSendDialbackKey(session.getStreamID().getID()); + return true; + } else { + LOG.debug("No authentication mechanism available."); + return false; + } + } + + // Handle dialback result + if ("db".equals(doc.getNamespacePrefix()) && "result".equals(rootTagName)) { + if ("valid".equals(doc.attributeValue("type"))) { + LOG.debug("Authentication succeeded!"); + LOG.debug("Dialback was successful."); + + connection.init(session); + isSessionAuthenticated = true; + // Set the remote domain name as the address of the session. + session.setAddress(new JID(null, domainPair.getRemote(), null)); + if (session instanceof LocalOutgoingServerSession) { + ((LocalOutgoingServerSession) session).setAuthenticationMethod(ServerSession.AuthenticationMethod.DIALBACK); + } + + return true; + } else { + LOG.debug("Dialback failed"); + LOG.debug("Failed to authenticate domain: the validation response was received, but did not grant authentication."); + return false; + } + } + + // Handles SASL failure + if ("failure".equals(rootTagName)) { + LOG.debug("EXTERNAL SASL failed."); + + // Try dialback + if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + LOG.debug("Trying to authenticate using dialback."); + LOG.debug("[Acting as Originating Server: Authenticate domain: " + domainPair.getLocal() + " with a RS in the domain of: " + domainPair.getRemote() + " (id: " + session.getStreamID() + ")]"); + ServerDialback dialback = new ServerDialback(connection, domainPair); + dialback.createAndSendDialbackKey(session.getStreamID().getID()); + return true; + } + + return false; + } + + // Handles SASL success + if ("success".equals(rootTagName)) { + LOG.debug("EXTERNAL SASL was successful."); + + // SASL was successful so initiate a new stream + StringBuilder sb = new StringBuilder(); + sb.append(""); + connection.deliverRawText(sb.toString()); + + connection.init(session); + isSessionAuthenticated = true; + // Set the remote domain name as the address of the session. + session.setAddress(new JID(null, domainPair.getRemote(), null)); + if (session instanceof LocalOutgoingServerSession) { + ((LocalOutgoingServerSession) session).setAuthenticationMethod(ServerSession.AuthenticationMethod.SASL_EXTERNAL); + } else { + LOG.debug("Expected session to be a LocalOutgoingServerSession but it isn't, unable to setAuthenticationMethod()."); + return false; + } + return true; + } + + // Handles proceed (prior to TLS negotiation) + if (rootTagName.equals("proceed")) { + LOG.debug("Received 'proceed' from remote server. Negotiating TLS..."); + try { + LOG.debug("Encrypting and authenticating connection ..."); + connection.startTLS(true, false); + } catch (Exception e) { + LOG.debug("TLS negotiation failed: " + e.getMessage()); + return false; + } + LOG.debug("TLS negotiation was successful. Connection encrypted. Proceeding with authentication..."); + + // Verify - TODO does this live here, should we do this when handling features mechanisms? + if (!SASLAuthentication.verifyCertificates(connection.getPeerCertificates(), domainPair.getRemote(), true)) { + if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + LOG.debug("Failed to verify certificates for SASL authentication. Will continue with dialback."); + // Will continue with dialback when the features stanza comes in and is processed (above) + } else { + LOG.warn("Unable to authenticate the connection: Failed to verify certificates for SASL authentication and dialback is not available."); + return false; + } + } + + LOG.debug("TLS negotiation was successful so initiate a new stream."); + StringBuilder sb = new StringBuilder(); + sb.append(""); + connection.deliverRawText(sb.toString()); + + return true; + } + + return false; + } + + private boolean shouldUseTls() { + return connection.getTlsPolicy() == Connection.TLSPolicy.optional || connection.getTlsPolicy() == Connection.TLSPolicy.required; + } + + private boolean mustUseTls() { + return connection.getTlsPolicy() == Connection.TLSPolicy.required; + } + + @Override + void startTLS() throws Exception { + connection.startTLS(true, false); + } + + @Override + String getNamespace() { + return "jabber:server"; + } + + @Override + boolean validateHost() { + return false; + } + + @Override + boolean validateJIDs() { + return false; + } + + public LocalSession getSession() { + return session; + } + + public boolean isSessionAuthenticated() { + return isSessionAuthenticated; + } + + @Override + void createSession(String serverName, XmlPullParser xpp, Connection connection) throws XmlPullParserException { + String currentStreamId = xpp.getAttributeValue("", "id"); + session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, BasicStreamIDFactory.createStreamID(currentStreamId)); + } + + + + +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java index f1633b8327..306995e785 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java @@ -684,6 +684,7 @@ public void deliverRawText(String text) { writeStarted(); writer.write(text); writer.flush(); + System.out.print("Sending: " + text); } catch (Exception e) { Log.debug("Error delivering raw text" + "\n" + this.toString(), e); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java index 3ac232c2b6..ed201b5b5a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java @@ -76,7 +76,7 @@ public abstract class StanzaHandler { // Flag that indicates that the client requested to use TLS and TLS has been negotiated. Once the // client sent a new initial stream header the value will return to false. - private boolean startedTLS = false; + protected boolean startedTLS = false; // Flag that indicates that the client requested to be authenticated. Once the // authentication process is over the value will return to false. protected boolean startedSASL = false; @@ -114,7 +114,7 @@ public void setSession(LocalSession session) { } public void process(String stanza, XMPPPacketReader reader) throws Exception { - if (!sessionCreated || isStartOfStream(stanza)) { + if (isStartOfStream(stanza) || !sessionCreated) { initiateSession(stanza, reader); } else { processStanza(stanza, reader); @@ -175,7 +175,7 @@ protected void processStanza(String stanza, XMPPPacketReader reader) throws Exce // only known case occurring 'in the wild' for this is Dialback, but it's valid XML / XMPP regardless). Re- // establishing those prefixes is achieved by wrapping the data-to-be-parsed in a dummy root element on which // the prefixes are defined. After the data has been parsed, the dummy root element is discarded. See OF-2556. - Log.trace("Connection defined namespace prefixes on its original 'stream' element."); + Log.debug("Connection defined namespace prefixes on its original 'stream' element."); final StringBuilder sb = new StringBuilder(); sb.append(" sb.append(" ").append(namespace.asXML())); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index b7eacd74dd..fe12acf0e7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -72,7 +72,7 @@ public class NettyConnection implements Connection { */ public static final String CHARSET = "UTF-8"; - private LocalSession session; + public LocalSession session; private ChannelHandlerContext channelHandlerContext; final private Map closeListeners = new HashMap<>(); @@ -325,7 +325,7 @@ public boolean isSecure() { @Override public boolean isEncrypted() { - return false; // TODO with netty + return channelHandlerContext.channel().pipeline().get(SSL_HANDLER_NAME) != null; } @Override @@ -365,7 +365,7 @@ public void deliver(Packet packet) throws UnauthorizedException { @Override public void deliverRawText(String text) { - //System.out.println("Sending: " + text); + System.out.println("Sending: " + text); if (!isClosed()) { boolean errorDelivering = false; ChannelFuture f = channelHandlerContext.writeAndFlush(text); @@ -391,22 +391,20 @@ public void deliverRawText(String text) { public void startTLS(boolean clientMode, boolean directTLS) throws Exception { final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); - SslContext sslContext = factory.createSslContext(); - - final SslHandler sslHandler = sslContext.newHandler(channelHandlerContext.alloc()); + final SslContext sslContext; if ( clientMode ) { -// filter = factory.createClientModeSslFilter(); + sslContext= factory.createClientModeSslContext(); } else { -// filter = factory.createServerModeSslFilter(); + sslContext = factory.createServerModeSslContext(); } + final SslHandler sslHandler = sslContext.newHandler(channelHandlerContext.alloc()); channelHandlerContext.pipeline().addFirst(SSL_HANDLER_NAME, sslHandler); if ( !clientMode && !directTLS ) { // Indicate the client that the server is ready to negotiate TLS deliverRawText( "" ); -// ioSession.getFilterChain().remove(STARTTLS_FILTER_NAME); } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index a063cdf01a..a36661f749 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -139,7 +139,7 @@ public void channelRead0(ChannelHandlerContext ctx, String message) { // updateReadBytesCounter(session); TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- - //System.out.println("RCVD: " + message); + System.out.println("Received: " + message); // Let the stanza handler process the received stanza try { ctx.channel().attr(HANDLER).get().process(message, parser); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java new file mode 100644 index 0000000000..748e261db1 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -0,0 +1,41 @@ +package org.jivesoftware.openfire.nio; + +import io.netty.channel.ChannelHandlerContext; +import org.dom4j.io.XMPPPacketReader; +import org.jivesoftware.openfire.*; +import org.jivesoftware.openfire.net.*; +import org.jivesoftware.openfire.session.*; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.slf4j.LoggerFactory; +import org.slf4j.Logger; + +public class NettyOutboundConnectionHandler extends NettyConnectionHandler { + private static final Logger log = LoggerFactory.getLogger(NettyOutboundConnectionHandler.class); + private final DomainPair domainPair; + + public NettyOutboundConnectionHandler(ConnectionConfiguration configuration, DomainPair domainPair) { + super(configuration); + this.domainPair = domainPair; + } + + @Override + NettyConnection createNettyConnection(ChannelHandlerContext ctx) { + return new NettyConnection(ctx, null, configuration); + } + + @Override + StanzaHandler createStanzaHandler(NettyConnection connection) { + return new RespondingServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection, domainPair ); + } + + @Override + int getMaxIdleTime() { + return 0; + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + System.out.println("Adding NettyOutboundConnectionHandler"); + super.handlerAdded(ctx); + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java index 9435f4fc4c..9860fb90fd 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -43,9 +43,14 @@ StanzaHandler createStanzaHandler(NettyConnection connection) { return new ServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection ); } + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + System.out.println("Adding NettyServerConnectionHandler"); + super.handlerAdded(ctx); + } - // TBD V +// TBD V diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java new file mode 100644 index 0000000000..f17b41e316 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -0,0 +1,145 @@ +package org.jivesoftware.openfire.nio; + +import io.netty.bootstrap.Bootstrap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.string.StringEncoder; +import org.jivesoftware.openfire.ConnectionManager; +import org.jivesoftware.openfire.XMPPServer; +import org.jivesoftware.openfire.net.RespondingServerStanzaHandler; +import org.jivesoftware.openfire.session.DomainPair; +import org.jivesoftware.openfire.session.LocalSession; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.jivesoftware.openfire.spi.ConnectionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + + +public class NettySessionInitializer { + + private static final Logger LOG = LoggerFactory.getLogger(NettySessionInitializer.class); + + private final DomainPair domainPair; + private final int port; + private EventLoopGroup workerGroup; + private final ExecutorService executor = Executors.newSingleThreadExecutor(); + + public NettySessionInitializer(DomainPair domainPair, int port) { + this.domainPair = domainPair; + this.port = port; + } + + +// TODO handle direct TLS +// if (directTLS) { +// try { +// connection.startTLS( true, true ); +// } catch ( SSLException ex ) { +// if ( JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && ex.getMessage().contains( "plaintext connection?" ) ) { +// Log.warn( "Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", socketAddress ); +// try { +// // Close old socket +// socket.close(); +// } catch ( Exception e ) { +// Log.debug( "An exception occurred (and is ignored) while trying to close a socket that was already in an error state.", e ); +// } +// socket = new Socket(); +// socket.connect( socketAddress, RemoteServerManager.getSocketTimeout() ); +// connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); +// directTLS = false; +// Log.info( "Re-established connection to {}. Proceeding without directTLS.", socketAddress ); +// } else { +// // Do not retry as non-DirectTLS, rethrow the exception. +// throw ex; +// } +// } +// } + + + + public Future init() { + workerGroup = new NioEventLoopGroup(); + try { + Bootstrap b = new Bootstrap(); + b.group(workerGroup); + b.channel(NioSocketChannel.class); + b.option(ChannelOption.SO_KEEPALIVE, true); + b.handler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) { + final ConnectionManager connectionManager = XMPPServer.getInstance().getConnectionManager(); + ConnectionConfiguration listenerConfiguration = connectionManager.getListener(ConnectionType.SOCKET_S2S, false).generateConnectionConfiguration(); + + ch.pipeline().addLast(new NettyXMPPDecoder()); + ch.pipeline().addLast(new StringEncoder()); + ch.pipeline().addLast(new NettyOutboundConnectionHandler(listenerConfiguration, domainPair)); + } + }); + + Channel channel = b.connect(domainPair.getRemote(), port).sync().channel(); + + // Start the session negotiation + sendOpeningStreamHeader(channel); + + return waitForSession(channel); + + + // TODO - do something here (block?) until we get a LocalOutgoingServerSession + // How do we report back that the session is open? + // This happens in RespondingServerStanzaHandler.createSession() + // Pass something in for ^ to update? + // + // How do Futures work? + // + // + // can we add a handler to the pipeline to handle this? + // Does this even make sense? Handlers deal with network in/out + +// return channel; + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + // TODO how do we shut down the server? +// workerGroup.shutdownGracefully(); + } + + } + + public void stop() { + workerGroup.shutdownGracefully(); + } + private Future waitForSession(Channel channel) { + RespondingServerStanzaHandler stanzaHandler = (RespondingServerStanzaHandler) channel.attr(NettyConnectionHandler.HANDLER).get(); + + return executor.submit(() -> { + while (!stanzaHandler.isSessionAuthenticated()) { + Thread.sleep(100); + } + return stanzaHandler.getSession(); + }); + } + + private void sendOpeningStreamHeader(Channel channel) { + LOG.debug("Send the stream header and wait for response..."); + StringBuilder sb = new StringBuilder(); + sb.append(""); + channel.writeAndFlush(sb.toString()); + System.out.println("Sending: " + sb.toString()); + } + +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java index 942b7c45a0..fd017d7468 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java @@ -16,24 +16,16 @@ package org.jivesoftware.openfire.server; -import java.lang.reflect.Field; -import java.time.Duration; -import java.time.temporal.ChronoUnit; -import java.util.*; -import java.util.concurrent.*; -import java.util.concurrent.locks.Lock; - import com.google.common.collect.Interner; import com.google.common.collect.Interners; import org.jivesoftware.openfire.RoutableChannelHandler; import org.jivesoftware.openfire.RoutingTable; -import org.jivesoftware.openfire.SessionManager; import org.jivesoftware.openfire.XMPPServer; -import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.cluster.NodeID; -import org.jivesoftware.openfire.interceptor.InterceptorManager; -import org.jivesoftware.openfire.interceptor.PacketRejectedException; -import org.jivesoftware.openfire.session.*; +import org.jivesoftware.openfire.session.ConnectionSettings; +import org.jivesoftware.openfire.session.DomainPair; +import org.jivesoftware.openfire.session.LocalOutgoingServerSession; +import org.jivesoftware.openfire.session.OutgoingServerSession; import org.jivesoftware.openfire.spi.RoutingTableImpl; import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.NamedThreadFactory; @@ -41,17 +33,16 @@ import org.jivesoftware.util.TaskEngine; import org.jivesoftware.util.cache.Cache; import org.jivesoftware.util.cache.CacheFactory; -import org.jivesoftware.util.cache.DefaultLocalCacheStrategy; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.xmpp.packet.IQ; -import org.xmpp.packet.JID; -import org.xmpp.packet.Message; -import org.xmpp.packet.Packet; -import org.xmpp.packet.PacketError; -import org.xmpp.packet.Presence; +import org.xmpp.packet.*; import javax.annotation.Nonnull; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.locks.Lock; /** * An OutgoingSessionPromise provides an asynchronic way for sending packets to remote servers. @@ -287,7 +278,7 @@ public void run() { Log.trace("Finished processing {}", domainPair); } - private RoutableChannelHandler establishConnection() throws Exception { // TODO: AG - Maybe here? + private RoutableChannelHandler establishConnection() throws Exception { Log.debug("Start establishing a connection for {}", domainPair); // Create a connection to the remote server from the domain where the packet has been sent boolean created; @@ -295,7 +286,7 @@ private RoutableChannelHandler establishConnection() throws Exception { // TODO: final Lock lock = serversCache.getLock(domainPair); lock.lock(); try { - created = LocalOutgoingServerSession.authenticateDomain(domainPair); //TODO: AG MAGIC HERE + created = LocalOutgoingServerSession.authenticateDomain(domainPair); // TODO: AG MAGIC HERE } finally { lock.unlock(); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java index 286a7956d1..abc62ee790 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java @@ -297,6 +297,16 @@ public LocalOutgoingServerSession createOutgoingSession(int port) { return null; } + /** + * Create a dialback key and send to receiving server + * + * @param id the stream id to be used for creating the dialback key. + */ + public void createAndSendDialbackKey(String id) { + String key = AuthFactory.createDigest( id, getSecretkey() ); + sendDialbackKey(key); + } + /** * Authenticates the Originating Server domain with the Receiving Server. Once the domain has * been authenticated the Receiving Server will start accepting packets from the Originating @@ -324,13 +334,7 @@ public boolean authenticateDomain(OutgoingServerSocketReader socketReader, Strin synchronized (socketReader) { log.debug( "Sending dialback key and wait for the validation response..." ); - StringBuilder sb = new StringBuilder(); - sb.append(""); - sb.append(key); - sb.append(""); - connection.deliverRawText(sb.toString()); + sendDialbackKey(key); // Process the answer from the Receiving Server try { @@ -362,6 +366,21 @@ else if ("db".equals(doc.getNamespacePrefix()) && "result".equals(doc.getName()) } } + /** + * Sends the supplied dialback key to receiving server + * + * @param key dialback key to send + */ + private void sendDialbackKey(String key) { + StringBuilder sb = new StringBuilder(); + sb.append(""); + sb.append(key); + sb.append(""); + connection.deliverRawText(sb.toString()); + } + /** * Returns a new {@link IncomingServerSession} with a domain validated by the Authoritative * Server. New domains may be added to the returned IncomingServerSession after they have diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index b043d5384a..c37c6080da 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -28,11 +28,11 @@ import org.jivesoftware.openfire.net.SASLAuthentication; import org.jivesoftware.openfire.net.SocketConnection; import org.jivesoftware.openfire.net.SocketUtil; +import org.jivesoftware.openfire.nio.NettySessionInitializer; import org.jivesoftware.openfire.server.OutgoingServerSocketReader; import org.jivesoftware.openfire.server.RemoteServerManager; import org.jivesoftware.openfire.server.ServerDialback; import org.jivesoftware.openfire.spi.BasicStreamIDFactory; -import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.StringUtils; import org.jivesoftware.util.TaskEngine; import org.slf4j.Logger; @@ -42,10 +42,7 @@ import org.xmpp.packet.*; import javax.annotation.Nonnull; -import javax.net.ssl.SSLException; -import javax.net.ssl.SSLHandshakeException; import java.io.IOException; -import java.io.InputStream; import java.io.InputStreamReader; import java.net.Socket; import java.net.SocketAddress; @@ -56,6 +53,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; /** @@ -85,7 +83,7 @@ public class LocalOutgoingServerSession extends LocalServerSession implements Ou private static final Interner remoteAuthMutex = Interners.newWeakInterner(); - private final OutgoingServerSocketReader socketReader; + private OutgoingServerSocketReader socketReader; private final Collection outgoingDomainPairs = new HashSet<>(); /** @@ -265,179 +263,22 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai Socket socket = socketToXmppDomain.getKey(); boolean directTLS = socketToXmppDomain.getValue(); - SocketConnection connection = null; + final SocketAddress socketAddress = socket.getRemoteSocketAddress(); + log.debug( "Opening a new connection to {} {}.", socketAddress, directTLS ? "using directTLS" : "that is initially not encrypted" ); + NettySessionInitializer sessionInitialiser = null; try { - final SocketAddress socketAddress = socket.getRemoteSocketAddress(); - log.debug( "Opening a new connection to {} {}.", socketAddress, directTLS ? "using directTLS" : "that is initially not encrypted" ); - connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); //TODO: AG NettyConnection? - if (directTLS) { - try { - connection.startTLS( true, true ); - } catch ( SSLException ex ) { - if ( JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && ex.getMessage().contains( "plaintext connection?" ) ) { - Log.warn( "Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", socketAddress ); - try { - // Close old socket - socket.close(); - } catch ( Exception e ) { - Log.debug( "An exception occurred (and is ignored) while trying to close a socket that was already in an error state.", e ); - } - socket = new Socket(); - socket.connect( socketAddress, RemoteServerManager.getSocketTimeout() ); - connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); - directTLS = false; - Log.info( "Re-established connection to {}. Proceeding without directTLS.", socketAddress ); - } else { - // Do not retry as non-DirectTLS, rethrow the exception. - throw ex; - } - } - } - - log.debug( "Send the stream header and wait for response..." ); - StringBuilder openingStream = new StringBuilder(); - openingStream.append(""); - connection.deliverRawText(openingStream.toString()); + // Wait for the future to give us a session... + sessionInitialiser = new NettySessionInitializer(domainPair, port); // Set a read timeout (of 5 seconds) so we don't keep waiting forever - int soTimeout = socket.getSoTimeout(); - socket.setSoTimeout(5000); - - XMPPPacketReader reader = new XMPPPacketReader(); - - final InputStream inputStream; - if (directTLS) { - inputStream = connection.getTLSStreamHandler().getInputStream(); - } else { - inputStream = socket.getInputStream(); - } - reader.getXPPParser().setInput(new InputStreamReader( inputStream, StandardCharsets.UTF_8 )); - - // Get the answer from the Receiving Server - XmlPullParser xpp = reader.getXPPParser(); - for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { - eventType = xpp.next(); - } - - String serverVersion = xpp.getAttributeValue("", "version"); - String id = xpp.getAttributeValue("", "id"); - log.debug( "Got a response (stream ID: {}, version: {}). Check if the remote server is XMPP 1.0 compliant...", id, serverVersion ); - - if (serverVersion != null && Session.decodeVersion(serverVersion)[0] >= 1) { - log.debug( "The remote server is XMPP 1.0 compliant (or at least reports to be)." ); - - // Restore default timeout - socket.setSoTimeout(soTimeout); - - Element features = reader.parseDocument().getRootElement(); - if (features != null) { - log.debug( "Processing stream features of the remote domain: {}", features.asXML() ); - if (directTLS) { - log.debug( "We connected to the remote server using direct TLS. Authenticate the connection with SASL..." ); - LocalOutgoingServerSession answer = authenticate(domainPair, connection, reader, openingStream, features, id); - if (answer != null) { - log.debug( "Successfully authenticated the connection with SASL)!" ); - // Everything went fine so return the encrypted and authenticated connection. - log.debug( "Successfully created new session!" ); - return answer; - } - log.debug( "Unable to authenticate the connection with SASL." ); - } else { - log.debug( "Check if both us as well as the remote server have enabled STARTTLS and/or dialback ..." ); - final boolean useTLS = connection.getTlsPolicy() == Connection.TLSPolicy.optional || connection.getTlsPolicy() == Connection.TLSPolicy.required; - if (useTLS && features.element("starttls") != null) { - log.debug( "Both us and the remote server support the STARTTLS feature. Encrypt and authenticate the connection with TLS & SASL..." ); - LocalOutgoingServerSession answer = encryptAndAuthenticate(domainPair, connection, reader, openingStream); - if (answer != null) { - log.debug( "Successfully encrypted/authenticated the connection with TLS/SASL)!" ); - // Everything went fine so return the secured and - // authenticated connection - log.debug( "Successfully created new session!" ); - return answer; - } - log.debug( "Unable to encrypt and authenticate the connection with TLS & SASL." ); - } - else if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - log.debug("I have no StartTLS yet I must TLS"); - connection.close(new StreamError(StreamError.Condition.not_authorized, "TLS is mandatory, but was not established.")); - return null; - } - // Check if we are going to try server dialback (XMPP 1.0) - else if (ServerDialback.isEnabled() && features.element("dialback") != null) { - log.debug( "Both us and the remote server support the 'dialback' feature. Authenticate the connection with dialback..." ); - ServerDialback method = new ServerDialback(connection, domainPair); - OutgoingServerSocketReader newSocketReader = new OutgoingServerSocketReader(reader); - if (method.authenticateDomain(newSocketReader, id)) { - log.debug( "Successfully authenticated the connection with dialback!" ); - StreamID streamID = BasicStreamIDFactory.createStreamID(id); - LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, newSocketReader, streamID); - connection.init(session); - session.setAuthenticationMethod(AuthenticationMethod.DIALBACK); - // Set the remote domain name as the address of the session. - session.setAddress(new JID(null, domainPair.getRemote(), null)); - log.debug( "Successfully created new session!" ); - return session; - } - else { - log.debug( "Unable to authenticate the connection with dialback." ); - } - } - } - } - else { - log.debug( "Error! No data from the remote server (expected a 'feature' element)."); - } - } else { - log.debug( "The remote server is not XMPP 1.0 compliant." ); - } - - log.debug( "Something went wrong so close the connection and try server dialback over a plain connection" ); - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - log.debug("I have no StartTLS yet I must TLS"); - connection.close(new StreamError(StreamError.Condition.not_authorized, "TLS is mandatory, but was not established.")); - return null; - } - connection.close(); - } - catch (SSLHandshakeException e) - { - // When not doing direct TLS but startTLS, this a failure as described in RFC6120, section 5.4.3.2 "STARTTLS Failure". - log.info( "{} negotiation failed. Closing connection (without sending any data such as or ).", (directTLS ? "Direct TLS" : "StartTLS" ), e ); - - // The receiving entity is expected to close the socket *without* sending any more data ( nor ). - // It is probably (see OF-794) best if we, as the initiating entity, therefor don't send any data either. - if (connection != null) { - connection.forceClose(); - - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - return null; - } - } - - if (e.getCause() instanceof CertificateException && JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { - log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled.", e); - return null; - } + // TODO extended timeout while working on dialback issue - which seems to have a 10s timeout + return (LocalOutgoingServerSession) sessionInitialiser.init().get(15000, TimeUnit.MILLISECONDS); } catch (Exception e) { // This might be RFC6120, section 5.4.2.2 "Failure Case" or even an unrelated problem. Handle 'normally'. log.warn( "An exception occurred while creating an encrypted session. Closing connection.", e ); - - if (connection != null) { - connection.close(); - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - return null; - } - } + if (sessionInitialiser != null) { sessionInitialiser.stop(); } } if (ServerDialback.isEnabled()) @@ -461,7 +302,7 @@ else if (ServerDialback.isEnabled() && features.element("dialback") != null) { } } - private static LocalOutgoingServerSession encryptAndAuthenticate(DomainPair domainPair, SocketConnection connection, XMPPPacketReader reader, StringBuilder openingStream) throws Exception { + public static LocalOutgoingServerSession encryptAndAuthenticate(DomainPair domainPair, SocketConnection connection, XMPPPacketReader reader, StringBuilder openingStream) throws Exception { final Logger log = LoggerFactory.getLogger(Log.getName() + "[Encrypt connection for: " + domainPair + "]" ); Element features; @@ -674,6 +515,10 @@ public LocalOutgoingServerSession(String localDomain, Connection connection, Out socketReader.setSession(this); } + public LocalOutgoingServerSession(String localDomain, Connection connection, StreamID streamID) { + super(localDomain, connection, streamID); + } + @Override boolean canProcess(Packet packet) { final DomainPair domainPair = new DomainPair(packet.getFrom().getDomain(), packet.getTo().getDomain()); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index 4ac623cceb..9a8aac8175 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -327,7 +327,7 @@ public synchronized SslContextFactory getSslContextFactory() } } - public SslContext createSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { + public SslContext createServerModeSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { getKeyManagers(); SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); @@ -346,11 +346,29 @@ public SslContext createSslContext() throws UnrecoverableKeyException, NoSuchAlg } builder.protocols(configuration.getEncryptionProtocols()); + builder.ciphers(configuration.getEncryptionCipherSuites()); builder.startTls(true); return builder.build(); } + public SslContext createClientModeSslContext() throws SSLException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException { + getKeyManagers(); + + // We will never send SSLV2 ClientHello messages + Set protocols = new HashSet<>(configuration.getEncryptionProtocols()); + protocols.remove("SSLv2Hello"); + + return SslContextBuilder + .forClient() + .protocols(protocols) + .ciphers(configuration.getEncryptionCipherSuites()) + .keyManager(keyManagerFactory) + .trustManager(getTrustManagers()[0]) + .startTls(false) + .build(); + } + /** * Creates an Apache MINA SslFilter that is configured to use server mode when handshaking. * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/RoutingTableImpl.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/RoutingTableImpl.java index 225348d1aa..7508056576 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/RoutingTableImpl.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/RoutingTableImpl.java @@ -613,7 +613,7 @@ private boolean routeToRemoteDomain(JID jid, Packet packet) { Log.trace("Routing to remote domain: {}", packet); // It is possible that serversCache has an entry for this domain, while the OutgoingSessionPromise is - // still processing it's queue. Stanzas must be delivered in order, so delivery of this stanza needs to + // still processing its queue. Stanzas must be delivered in order, so delivery of this stanza needs to // be postponed until after the queue is empty (OF-2321). // // The code block below is guarded by the mutex that also guards: @@ -622,7 +622,7 @@ private boolean routeToRemoteDomain(JID jid, Packet packet) { // This will ensure that a new stanza: // - is supplied to OutgoingSessionPromise before queue processing is started / can start. // - is not supplied to OutgoingSessionPromise after queue processing has started. In that case, this code will - // block until until the OutgoingSessionPromise is done processing, and will then route the stanza through the localRoutingTable. + // block until the OutgoingSessionPromise is done processing, and will then route the stanza through the localRoutingTable. synchronized (OutgoingSessionPromise.getInstance().getMutex(domainPair)) { if (OutgoingSessionPromise.getInstance().hasProcess(domainPair)) { From c8a2ba24635052545a4afc04f1feb7b8a7d75078 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Thu, 13 Jul 2023 13:09:26 +0100 Subject: [PATCH 11/56] OF-2559 - Direct TLS on Netty inbound and outbound S2S --- .../openfire/net/ServerStanzaHandler.java | 7 +- .../openfire/nio/NettyConnection.java | 2 +- .../openfire/nio/NettyConnectionHandler.java | 2 +- .../nio/NettyServerConnectionHandler.java | 5 +- .../openfire/nio/NettySessionInitializer.java | 113 ++++++++---------- .../openfire/nio/ServerConnectionHandler.java | 5 +- .../session/LocalOutgoingServerSession.java | 2 +- .../spi/EncryptionArtifactFactory.java | 4 +- .../openfire/spi/NettyConnectionAcceptor.java | 2 +- .../openfire/spi/NettyServerInitializer.java | 11 +- 10 files changed, 80 insertions(+), 73 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java index dc9bf55d10..7d74d1b494 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java @@ -62,8 +62,11 @@ public class ServerStanzaHandler extends StanzaHandler { .setDynamic(true) .build(); - public ServerStanzaHandler(PacketRouter router, Connection connection) { + private final boolean directTLS; + + public ServerStanzaHandler(PacketRouter router, Connection connection, boolean directTLS) { super(router, connection); + this.directTLS = directTLS; } @Override @@ -115,7 +118,7 @@ void createSession(String serverName, XmlPullParser xpp, Connection connection) { // The connected client is a server so create an IncomingServerSession try { - session = LocalIncomingServerSession.createSession(serverName, xpp, connection, false); + session = LocalIncomingServerSession.createSession(serverName, xpp, connection, this.directTLS); } catch (IOException e) { Log.error(e.getMessage(), e); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index fe12acf0e7..88b897d5a1 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -396,7 +396,7 @@ public void startTLS(boolean clientMode, boolean directTLS) throws Exception { if ( clientMode ) { sslContext= factory.createClientModeSslContext(); } else { - sslContext = factory.createServerModeSslContext(); + sslContext = factory.createServerModeSslContext(directTLS); } final SslHandler sslHandler = sslContext.newHandler(channelHandlerContext.alloc()); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index a36661f749..1df5ac4699 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -46,7 +46,7 @@ public abstract class NettyConnectionHandler extends SimpleChannelInboundHandler private static final Logger Log = LoggerFactory.getLogger(NettyConnectionHandler.class); static final AttributeKey XML_PARSER = AttributeKey.valueOf("XML-PARSER"); - static final AttributeKey CONNECTION = AttributeKey.valueOf("CONNECTION"); + public static final AttributeKey CONNECTION = AttributeKey.valueOf("CONNECTION"); static final AttributeKey HANDLER = AttributeKey.valueOf("HANDLER"); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java index 9860fb90fd..0981eb077c 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -1,6 +1,7 @@ package org.jivesoftware.openfire.nio; import io.netty.channel.ChannelHandlerContext; +import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.PacketDeliverer; import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.ServerStanzaHandler; @@ -26,10 +27,12 @@ public class NettyServerConnectionHandler extends NettyConnectionHandler .setDefaultValue(true) .setDynamic(true) .build(); + private final boolean directTLS; public NettyServerConnectionHandler(ConnectionConfiguration configuration) { super(configuration); + this.directTLS = configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode; } @Override @@ -40,7 +43,7 @@ NettyConnection createNettyConnection(ChannelHandlerContext ctx) { @Override StanzaHandler createStanzaHandler(NettyConnection connection) { - return new ServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection ); + return new ServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection, directTLS); } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java index f17b41e316..5abfc84ffb 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -1,10 +1,7 @@ package org.jivesoftware.openfire.nio; import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelInitializer; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; +import io.netty.channel.*; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioSocketChannel; @@ -12,17 +9,23 @@ import org.jivesoftware.openfire.ConnectionManager; import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.RespondingServerStanzaHandler; +import org.jivesoftware.openfire.session.ConnectionSettings; import org.jivesoftware.openfire.session.DomainPair; import org.jivesoftware.openfire.session.LocalSession; import org.jivesoftware.openfire.spi.ConnectionConfiguration; import org.jivesoftware.openfire.spi.ConnectionType; +import org.jivesoftware.util.JiveGlobals; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.net.ssl.SSLException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import static org.jivesoftware.openfire.nio.NettyConnectionHandler.CONNECTION; +import static org.jivesoftware.openfire.session.Session.Log; + public class NettySessionInitializer { @@ -30,42 +33,21 @@ public class NettySessionInitializer { private final DomainPair domainPair; private final int port; + private boolean directTLS = false; private EventLoopGroup workerGroup; private final ExecutorService executor = Executors.newSingleThreadExecutor(); + private Channel channel; + + public NettySessionInitializer(DomainPair domainPair, int port, boolean directTLS) { + this(domainPair, port); + this.directTLS = directTLS; + } public NettySessionInitializer(DomainPair domainPair, int port) { this.domainPair = domainPair; this.port = port; } - -// TODO handle direct TLS -// if (directTLS) { -// try { -// connection.startTLS( true, true ); -// } catch ( SSLException ex ) { -// if ( JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && ex.getMessage().contains( "plaintext connection?" ) ) { -// Log.warn( "Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", socketAddress ); -// try { -// // Close old socket -// socket.close(); -// } catch ( Exception e ) { -// Log.debug( "An exception occurred (and is ignored) while trying to close a socket that was already in an error state.", e ); -// } -// socket = new Socket(); -// socket.connect( socketAddress, RemoteServerManager.getSocketTimeout() ); -// connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); -// directTLS = false; -// Log.info( "Re-established connection to {}. Proceeding without directTLS.", socketAddress ); -// } else { -// // Do not retry as non-DirectTLS, rethrow the exception. -// throw ex; -// } -// } -// } - - - public Future init() { workerGroup = new NioEventLoopGroup(); try { @@ -74,49 +56,58 @@ public Future init() { b.channel(NioSocketChannel.class); b.option(ChannelOption.SO_KEEPALIVE, true); b.handler(new ChannelInitializer() { - @Override - public void initChannel(SocketChannel ch) { - final ConnectionManager connectionManager = XMPPServer.getInstance().getConnectionManager(); - ConnectionConfiguration listenerConfiguration = connectionManager.getListener(ConnectionType.SOCKET_S2S, false).generateConnectionConfiguration(); - - ch.pipeline().addLast(new NettyXMPPDecoder()); - ch.pipeline().addLast(new StringEncoder()); - ch.pipeline().addLast(new NettyOutboundConnectionHandler(listenerConfiguration, domainPair)); - } + @Override + public void initChannel(SocketChannel ch) throws Exception { + final ConnectionManager connectionManager = XMPPServer.getInstance().getConnectionManager(); + ConnectionConfiguration listenerConfiguration = connectionManager.getListener(ConnectionType.SOCKET_S2S, false).generateConnectionConfiguration(); + + ch.pipeline().addLast(new NettyXMPPDecoder()); + ch.pipeline().addLast(new StringEncoder()); + ch.pipeline().addLast(new NettyOutboundConnectionHandler(listenerConfiguration, domainPair)); + // Should have a connection + if (directTLS) { + ch.attr(CONNECTION).get().startTLS(true, true); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + super.exceptionCaught(ctx, cause); + if (exceptionOccurredForDirectTLS(cause)) { + if ( directTLS && + JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && + cause.getMessage().contains( "plaintext connection?") + ) { + Log.warn( "Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", domainPair.getRemote() ); + directTLS = false; + Log.info( "Re-establishing connection to {}. Proceeding without directTLS.", domainPair.getRemote() ); + init(); + } + } + } + + public boolean exceptionOccurredForDirectTLS(Throwable cause) { + return cause instanceof SSLException; + } }); - Channel channel = b.connect(domainPair.getRemote(), port).sync().channel(); + this.channel = b.connect(domainPair.getRemote(), port).sync().channel(); // Start the session negotiation sendOpeningStreamHeader(channel); return waitForSession(channel); - - - // TODO - do something here (block?) until we get a LocalOutgoingServerSession - // How do we report back that the session is open? - // This happens in RespondingServerStanzaHandler.createSession() - // Pass something in for ^ to update? - // - // How do Futures work? - // - // - // can we add a handler to the pipeline to handle this? - // Does this even make sense? Handlers deal with network in/out - -// return channel; } catch (InterruptedException e) { - throw new RuntimeException(e); - } finally { - // TODO how do we shut down the server? -// workerGroup.shutdownGracefully(); + stop(); + throw new RuntimeException(e); // TODO: Better to throw all exceptions and catch outside? } - } public void stop() { + channel.close(); workerGroup.shutdownGracefully(); } + private Future waitForSession(Channel channel) { RespondingServerStanzaHandler stanzaHandler = (RespondingServerStanzaHandler) channel.attr(NettyConnectionHandler.HANDLER).get(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java index 30034fbac0..cdcd3a7e75 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java @@ -1,6 +1,7 @@ package org.jivesoftware.openfire.nio; import org.apache.mina.core.session.IoSession; +import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.PacketDeliverer; import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.ServerStanzaHandler; @@ -26,10 +27,12 @@ public class ServerConnectionHandler extends ConnectionHandler .setDefaultValue(true) .setDynamic(true) .build(); + private final boolean directTLS; public ServerConnectionHandler( ConnectionConfiguration configuration ) { super( configuration ); + this.directTLS = configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode; } @Override @@ -42,7 +45,7 @@ NIOConnection createNIOConnection( IoSession session ) @Override StanzaHandler createStanzaHandler( NIOConnection connection ) { - return new ServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection ); + return new ServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection, directTLS ); } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index c37c6080da..4af38c173a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -269,7 +269,7 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai try { // Wait for the future to give us a session... - sessionInitialiser = new NettySessionInitializer(domainPair, port); + sessionInitialiser = new NettySessionInitializer(domainPair, port, directTLS); // Set a read timeout (of 5 seconds) so we don't keep waiting forever // TODO extended timeout while working on dialback issue - which seems to have a 10s timeout return (LocalOutgoingServerSession) sessionInitialiser.init().get(15000, TimeUnit.MILLISECONDS); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index 9a8aac8175..fd04677537 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -327,7 +327,7 @@ public synchronized SslContextFactory getSslContextFactory() } } - public SslContext createServerModeSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { + public SslContext createServerModeSslContext(boolean directTLS) throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { getKeyManagers(); SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); @@ -347,7 +347,7 @@ public SslContext createServerModeSslContext() throws UnrecoverableKeyException, builder.protocols(configuration.getEncryptionProtocols()); builder.ciphers(configuration.getEncryptionCipherSuites()); - builder.startTls(true); + builder.startTls(!directTLS); return builder.build(); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 960c7e6e4e..8d39cdaf74 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -104,7 +104,7 @@ public synchronized void start() { // Instantiate a new Channel to accept incoming connections. .channel(NioServerSocketChannel.class) // The handler specified here will always be evaluated by a newly accepted Channel. - .childHandler(new NettyServerInitializer(connectionHandler)) + .childHandler(new NettyServerInitializer(connectionHandler, configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode)) // Set the listen backlog (queue) length. .option(ChannelOption.SO_BACKLOG, JiveGlobals.getIntProperty("xmpp.socket.backlog", 50)) // option() is for the NioServerSocketChannel that accepts incoming connections. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java index 773496ddb4..f0f13d081a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java @@ -12,6 +12,8 @@ import java.time.Duration; import java.time.temporal.ChronoUnit; +import static org.jivesoftware.openfire.nio.NettyConnectionHandler.CONNECTION; + /** * Creates a newly configured {@link ChannelPipeline} for a new channel. */ @@ -28,17 +30,22 @@ public class NettyServerInitializer extends ChannelInitializer { .build(); private final NettyConnectionHandler businessLogicHandler; + private final boolean directTLS; - public NettyServerInitializer(NettyConnectionHandler businessLogicHandler) { + public NettyServerInitializer(NettyConnectionHandler businessLogicHandler, boolean directTLS) { this.businessLogicHandler = businessLogicHandler; + this.directTLS = directTLS; } @Override - public void initChannel(SocketChannel ch) { + public void initChannel(SocketChannel ch) throws Exception { ch.pipeline().addLast(new NettyXMPPDecoder()); ch.pipeline().addLast(new StringEncoder()); ch.pipeline().addLast("stalledSessionHandler", new WriteTimeoutHandler(Math.toIntExact(WRITE_TIMEOUT_SECONDS.getValue().getSeconds()))); ch.pipeline().addLast(businessLogicHandler); + if (directTLS) { + ch.attr(CONNECTION).get().startTLS(false, true); + } } } From c52100fb51bf82ca5e2705ddbb6711224d769593 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 17 Jul 2023 14:18:06 +0100 Subject: [PATCH 12/56] OF-2559 Refactor to remove duplication --- .../session/LocalIncomingServerSession.java | 115 +----------------- 1 file changed, 1 insertion(+), 114 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java index ff26cf0a0b..df5fad0835 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java @@ -110,120 +110,9 @@ public class LocalIncomingServerSession extends LocalServerSession implements In */ public static LocalIncomingServerSession createSession(String serverName, XMPPPacketReader reader, SocketConnection connection, boolean directTLS) throws XmlPullParserException, IOException { - XmlPullParser xpp = reader.getXPPParser(); - - String version = xpp.getAttributeValue("", "version"); - String fromDomain = xpp.getAttributeValue("", "from"); - String toDomain = xpp.getAttributeValue("", "to"); - int[] serverVersion = version != null ? Session.decodeVersion(version) : new int[] {0,0}; - - if (toDomain == null) { - toDomain = serverName; - } - - // Retrieve list of namespaces declared in current element (OF-2556) - connection.setAdditionalNamespaces(XMPPPacketReader.getPrefixedNamespacesOnCurrentElement(xpp)); - - try { - // Get the stream ID for the new session - StreamID streamID = SessionManager.getInstance().nextStreamID(); - // Create a server Session for the remote server - LocalIncomingServerSession session = SessionManager.getInstance().createIncomingServerSession(connection, streamID, fromDomain); - Log.debug("Creating new session with stream ID '{}' for local '{}' to peer '{}'.", streamID, toDomain, fromDomain); - - // Send the stream header - StringBuilder openingStream = new StringBuilder(); - openingStream.append("= 1) { - openingStream.append(" version=\"1.0\">"); - } else { - openingStream.append('>'); - } - - Log.trace("Outbound opening stream: {}", openingStream); - connection.deliverRawText(openingStream.toString()); - - if (serverVersion[0] >= 1) { - // Remote server is XMPP 1.0 compliant so offer TLS and SASL to establish the connection (and server dialback) - - // Indicate the TLS policy to use for this connection - Connection.TLSPolicy tlsPolicy = connection.getTlsPolicy(); - boolean hasCertificates = false; - try { - hasCertificates = XMPPServer.getInstance().getCertificateStoreManager().getIdentityStore( ConnectionType.SOCKET_S2S ).getStore().size() > 0; - } - catch (Exception e) { - Log.error(e.getMessage(), e); - } - if (Connection.TLSPolicy.required == tlsPolicy && !hasCertificates) { - Log.error("Server session rejected. TLS is required but no certificates " + - "were created."); - return null; - } - connection.setTlsPolicy(hasCertificates ? tlsPolicy : Connection.TLSPolicy.disabled); - } - - // Indicate the compression policy to use for this connection - connection.setCompressionPolicy( connection.getConfiguration().getCompressionPolicy() ); - - StringBuilder sb = new StringBuilder(); - - if (serverVersion[0] >= 1) { - Log.trace("Remote server is XMPP 1.0 compliant so offer TLS and SASL to establish the connection (and server dialback)"); - - sb.append(""); - - if (!directTLS && (connection.getTlsPolicy() == Connection.TLSPolicy.required || connection.getTlsPolicy() == Connection.TLSPolicy.optional)) { - sb.append(""); - if (!ServerDialback.isEnabled()) { - Log.debug("Server dialback is disabled so TLS is required"); - sb.append(""); - } - sb.append(""); - } - - // Include available SASL Mechanisms - sb.append(SASLAuthentication.getSASLMechanisms(session)); - - if (ServerDialback.isEnabled()) { - // Also offer server dialback (when TLS is not required). Server dialback may be offered - // after TLS has been negotiated and a self-signed certificate is being used - sb.append(""); - } - - sb.append(""); - } else { - Log.debug("Don't offer stream-features to pre-1.0 servers, as it confuses them. Sending features to Openfire < 3.7.1 confuses it too - OF-443)"); - } - - Log.trace("Outbound feature advertisement: {}", sb); - connection.deliverRawText(sb.toString()); - - Log.trace("Set the domain or subdomain of the local server targeted by the remote server: {}", serverName); - session.setLocalDomain(serverName); - return session; - } - catch (Exception e) { - Log.error("Error establishing connection from remote server: {}", connection, e); - connection.close(new StreamError(StreamError.Condition.internal_server_error)); - return null; - } + return createSession(serverName, reader.getXPPParser(), connection, directTLS); } - public static LocalIncomingServerSession createSession(String serverName, XmlPullParser xpp, Connection connection, boolean directTLS) throws XmlPullParserException, IOException { @@ -336,8 +225,6 @@ public static LocalIncomingServerSession createSession(String serverName, XmlPul } } - - public LocalIncomingServerSession(String serverName, Connection connection, StreamID streamID, String fromDomain) { super(serverName, connection, streamID); this.fromDomain = fromDomain; From ee5bd079a2e5d9ba6d14714b6b0adab0054f74a5 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 17 Jul 2023 14:19:46 +0100 Subject: [PATCH 13/56] OF-2559 Remove temporary long timeout Increased time allowed dialback DNS lookup to timeout and enabled us to debug deeper into the flow. --- .../openfire/session/LocalOutgoingServerSession.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index 4af38c173a..ca4085e366 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -271,8 +271,7 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai // Wait for the future to give us a session... sessionInitialiser = new NettySessionInitializer(domainPair, port, directTLS); // Set a read timeout (of 5 seconds) so we don't keep waiting forever - // TODO extended timeout while working on dialback issue - which seems to have a 10s timeout - return (LocalOutgoingServerSession) sessionInitialiser.init().get(15000, TimeUnit.MILLISECONDS); + return (LocalOutgoingServerSession) sessionInitialiser.init().get(5000, TimeUnit.MILLISECONDS); } catch (Exception e) { From 3f6241fffa9394b0b379d3d699d5496e3f416832 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 17 Jul 2023 14:23:32 +0100 Subject: [PATCH 14/56] OF-2559 Tidy up, removed unused properties --- .../openfire/nio/NettyConnection.java | 36 +++---------------- 1 file changed, 4 insertions(+), 32 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 88b897d5a1..33adb28bde 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -43,15 +43,11 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.net.UnknownHostException; -import java.nio.charset.CharsetEncoder; -import java.nio.charset.CodingErrorAction; -import java.nio.charset.StandardCharsets; import java.security.cert.Certificate; import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; import static com.jcraft.jzlib.JZlib.Z_BEST_COMPRESSION; @@ -65,7 +61,7 @@ public class NettyConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(NettyConnection.class); private static final String SSL_HANDLER_NAME = "ssl"; - private ConnectionConfiguration configuration; + private final ConnectionConfiguration configuration; /** * The utf-8 charset for decoding and encoding XMPP packet streams. @@ -73,7 +69,7 @@ public class NettyConnection implements Connection { public static final String CHARSET = "UTF-8"; public LocalSession session; - private ChannelHandlerContext channelHandlerContext; + private final ChannelHandlerContext channelHandlerContext; final private Map closeListeners = new HashMap<>(); @@ -81,10 +77,9 @@ public class NettyConnection implements Connection { * Deliverer to use when the connection is closed or was closed when delivering * a packet. */ - private PacketDeliverer backupDeliverer; + private final PacketDeliverer backupDeliverer; private int majorVersion = 1; private int minorVersion = 0; - private String language = null; /** * TLS policy currently in use for this connection. @@ -96,7 +91,6 @@ public class NettyConnection implements Connection { * Compression policy currently in use for this connection. */ private CompressionPolicy compressionPolicy = CompressionPolicy.disabled; - private static final ThreadLocal encoder = new ThreadLocalEncoder(); /** * Flag that specifies if the connection should be considered closed. Closing a NIO connection @@ -104,18 +98,7 @@ public class NettyConnection implements Connection { * keep this flag to avoid using the connection between #close was used and the socket is actually * closed. */ - private AtomicReference state = new AtomicReference<>(State.OPEN); - - /** - * Lock used to ensure the integrity of the underlying IoSession (refer to - * https://issues.apache.org/jira/browse/DIRMINA-653 for details) - *

- * This lock can be removed once Openfire guarantees a stable delivery - * order, in which case {@link #deliver(Packet)} won't be called - * concurrently any more, which made this lock necessary in the first place. - *

- */ - private final ReentrantLock ioSessionLock = new ReentrantLock(true); + private final AtomicReference state = new AtomicReference<>(State.OPEN); public NettyConnection(ChannelHandlerContext channelHandlerContext, @Nullable PacketDeliverer packetDeliverer, ConnectionConfiguration configuration ) { this.channelHandlerContext = channelHandlerContext; @@ -123,7 +106,6 @@ public NettyConnection(ChannelHandlerContext channelHandlerContext, @Nullable Pa this.configuration = configuration; } - @Override public boolean validate() { if (isClosed()) { @@ -311,7 +293,6 @@ public void reinit(LocalSession owner) { } } - @Override public boolean isClosed() { return state.get() == State.CLOSED; @@ -473,13 +454,4 @@ public String toString() { return super.toString() + " Netty Session: " + channelHandlerContext.name(); } - private static class ThreadLocalEncoder extends ThreadLocal { - - @Override - protected CharsetEncoder initialValue() { - return StandardCharsets.UTF_8.newEncoder() - .onMalformedInput(CodingErrorAction.REPORT) - .onUnmappableCharacter(CodingErrorAction.REPORT); - } - } } From dec32a3dba17b05f2a9fc80143bc3518ce9d9584 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 17 Jul 2023 14:25:09 +0100 Subject: [PATCH 15/56] OF-2559 Tidy up comments --- .../org/jivesoftware/admin/PluginFilter.java | 8 ++-- .../net/RespondingServerStanzaHandler.java | 26 ++++++++++--- .../openfire/net/SocketConnection.java | 4 +- .../jivesoftware/openfire/net/SocketUtil.java | 4 +- .../openfire/net/StanzaHandler.java | 2 +- .../openfire/nio/ConnectionHandler.java | 4 +- .../nio/NettyClientConnectionHandler.java | 24 ++++++++++++ .../openfire/nio/NettyConnectionHandler.java | 9 +++-- .../nio/NettyOutboundConnectionHandler.java | 37 +++++++++++++++---- .../nio/NettyServerConnectionHandler.java | 28 ++++++++++---- .../openfire/nio/NettySessionInitializer.java | 20 +++++++++- .../openfire/nio/NettyXMPPDecoder.java | 9 +++-- .../openfire/nio/ServerConnectionHandler.java | 16 ++++++++ .../openfire/nio/XMLLightweightParser.java | 6 +-- .../openfire/nio/XMPPDecoder.java | 2 +- .../server/OutgoingSessionPromise.java | 8 ++-- .../openfire/spi/ConnectionListener.java | 5 +-- .../openfire/spi/NettyConnectionAcceptor.java | 34 ++++++++++------- 18 files changed, 181 insertions(+), 65 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java index d8696773e2..924bb4ff9c 100644 --- a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java +++ b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2005-2008 Jive Software, Ignite Realtime Foundation 2022. All rights reserved. + * Copyright (C) 2005-2008 Jive Software, 2022-2023 Ignite Realtime Foundation. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -29,7 +29,7 @@ import java.util.concurrent.ConcurrentHashMap; /** - * A servlet filter that plugin classes can use to dynamically register and un-register filter logic. + * A servlet filter that plugin classes can use to dynamically register and un-register filter logic.

* * This implementation assumes, but does not enforce, that filters installed by plugins are applied to URL patterns that * match the plugin. When filters installed by different plugins are applied to the same URL, the behavior of this @@ -47,10 +47,10 @@ public class PluginFilter implements Filter { /** * Adds a filter to the list of filters that will be run on every request of which the URL matches the URL that * is registered with this filter. More specifically, the request URL should be equal to, or start with, the filter - * URL. + * URL.

* * Multiple filters can be registered on the same URL, in which case they will be executed in the order in which - * they were added. + * they were added.

* * Adding a filter does not initialize the plugin instance. * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index 680d6a88ba..8146deefd3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -1,3 +1,19 @@ +/* + * Copyright (C) 2005-2008 Jive Software, 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.net; import org.dom4j.DocumentException; @@ -9,7 +25,10 @@ import org.jivesoftware.openfire.PacketRouter; import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.server.ServerDialback; -import org.jivesoftware.openfire.session.*; +import org.jivesoftware.openfire.session.DomainPair; +import org.jivesoftware.openfire.session.LocalOutgoingServerSession; +import org.jivesoftware.openfire.session.LocalSession; +import org.jivesoftware.openfire.session.ServerSession; import org.jivesoftware.openfire.spi.BasicStreamIDFactory; import org.jivesoftware.util.StringUtils; import org.slf4j.Logger; @@ -18,7 +37,6 @@ import org.xmlpull.v1.XmlPullParserException; import org.xmpp.packet.JID; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -296,8 +314,4 @@ void createSession(String serverName, XmlPullParser xpp, Connection connection) String currentStreamId = xpp.getAttributeValue("", "id"); session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, BasicStreamIDFactory.createStreamID(currentStreamId)); } - - - - } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java index 306995e785..4348e296a6 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java @@ -68,7 +68,7 @@ public class SocketConnection implements Connection { * Milliseconds a connection has to be idle to be closed. Timeout is disabled by default. It's * up to the connection's owner to configure the timeout value. Sending stanzas to the client * is not considered as activity. We are only considering the connection active when the - * client sends some data or hearbeats (i.e. whitespaces) to the server. + * client sends some data or heartbeats (i.e. whitespaces) to the server. * The reason for this is that sending data will fail if the connection is closed. And if * the thread is blocked while sending data (because the socket is closed) then the clean up * thread will close the socket anyway. @@ -684,7 +684,7 @@ public void deliverRawText(String text) { writeStarted(); writer.write(text); writer.flush(); - System.out.print("Sending: " + text); + Log.trace("Sending: " + text); } catch (Exception e) { Log.debug("Error delivering raw text" + "\n" + this.toString(), e); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java index 0abe398a90..02abf3ab0a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java @@ -69,10 +69,10 @@ public static Map.Entry createSocketToXmppDomain( String xmppDo try { // (re)initialize the socket. - socket = new Socket(); // AG HERE + socket = new Socket(); Log.debug( "Trying to create socket connection to XMPP domain '{}' using remote host: {}:{} (blocks up to {} ms) ...", xmppDomain, realHostname, realPort, socketTimeout ); - socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); // AG HERE + socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); Log.debug( "Successfully created socket connection to XMPP domain '{}' using remote host: {}:{}!", xmppDomain, realHostname, realPort ); return new AbstractMap.SimpleEntry<>(socket, directTLS); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java index ed201b5b5a..6ac18962ff 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java @@ -175,7 +175,7 @@ protected void processStanza(String stanza, XMPPPacketReader reader) throws Exce // only known case occurring 'in the wild' for this is Dialback, but it's valid XML / XMPP regardless). Re- // establishing those prefixes is achieved by wrapping the data-to-be-parsed in a dummy root element on which // the prefixes are defined. After the data has been parsed, the dummy root element is discarded. See OF-2556. - Log.debug("Connection defined namespace prefixes on its original 'stream' element."); + Log.trace("Connection defined namespace prefixes on its original 'stream' element."); final StringBuilder sb = new StringBuilder(); sb.append(" sb.append(" ").append(namespace.asXML())); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java index d89c9330f5..92d6db7e79 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2005-2008 Jive Software, 2022 Ignite Realtime Foundation. All rights reserved. + * Copyright (C) 2005-2008 Jive Software, 2022-2023 Ignite Realtime Foundation. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -33,8 +33,6 @@ import org.xmlpull.v1.XmlPullParserFactory; import org.xmpp.packet.StreamError; -import java.nio.charset.StandardCharsets; - /** * A ConnectionHandler is responsible for creating new sessions, destroying sessions and delivering * received XML stanzas to the proper StanzaHandler. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java index 1a009a84b2..ea49b67d0e 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java @@ -1,3 +1,19 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.nio; import io.netty.channel.ChannelHandlerContext; @@ -11,6 +27,13 @@ import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.SystemProperty; +/** + * Client-specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created + * and how to build and configure a {@link NettyConnection}. + * + * @author Matthew Vivian + * @author Alex Gidman + */ public class NettyClientConnectionHandler extends NettyConnectionHandler{ /** @@ -30,6 +53,7 @@ public NettyClientConnectionHandler(ConnectionConfiguration configuration) { @Override NettyConnection createNettyConnection(ChannelHandlerContext ctx) { + // TODO - can this be moved up to superclass? appears to be same as Server implementation final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; return new NettyConnection(ctx, backupDeliverer, configuration); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index 1df5ac4699..551f21bb08 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2005-2008 Jive Software, 2022 Ignite Realtime Foundation. All rights reserved. + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -32,11 +32,12 @@ import org.xmlpull.v1.XmlPullParserFactory; import org.xmpp.packet.StreamError; -import java.nio.charset.StandardCharsets; - /** * A NettyConnectionHandler is responsible for creating new sessions, destroying sessions and delivering - * received XML stanzas to the proper StanzaHandler. + * received XML stanzas to the proper StanzaHandler.

+ * + * Subclasses of this will supply a specific {@link StanzaHandler} implementation depending on the + * type of connection to be handled, e.g. C2S, S2S. * * @author Matthew Vivian * @author Alex Gidman diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java index 748e261db1..6d8ec93ef3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -1,16 +1,39 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.nio; import io.netty.channel.ChannelHandlerContext; -import org.dom4j.io.XMPPPacketReader; -import org.jivesoftware.openfire.*; -import org.jivesoftware.openfire.net.*; -import org.jivesoftware.openfire.session.*; +import org.jivesoftware.openfire.XMPPServer; +import org.jivesoftware.openfire.net.RespondingServerStanzaHandler; +import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.session.DomainPair; import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.slf4j.LoggerFactory; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * Outbound (S2S) specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created + * and how to build and configure a {@link NettyConnection}. + * + * @author Matthew Vivian + * @author Alex Gidman + */ public class NettyOutboundConnectionHandler extends NettyConnectionHandler { - private static final Logger log = LoggerFactory.getLogger(NettyOutboundConnectionHandler.class); + private static final Logger Log = LoggerFactory.getLogger(NettyOutboundConnectionHandler.class); private final DomainPair domainPair; public NettyOutboundConnectionHandler(ConnectionConfiguration configuration, DomainPair domainPair) { @@ -35,7 +58,7 @@ int getMaxIdleTime() { @Override public void handlerAdded(ChannelHandlerContext ctx) { - System.out.println("Adding NettyOutboundConnectionHandler"); + Log.trace("Adding NettyOutboundConnectionHandler"); super.handlerAdded(ctx); } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java index 0981eb077c..87f7f616e3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -1,3 +1,19 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.nio; import io.netty.channel.ChannelHandlerContext; @@ -11,10 +27,11 @@ import org.jivesoftware.util.SystemProperty; /** - * ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created and how to build and configure - * a {@link NettyConnection}. + * Server-specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created + * and how to build and configure a {@link NettyConnection}. * - * @author Guus der Kinderen, guus.der.kinderen@gmail.com + * @author Matthew Vivian + * @author Alex Gidman */ public class NettyServerConnectionHandler extends NettyConnectionHandler { @@ -53,10 +70,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { } -// TBD V - - - + // TODO - check idle time is set for connections (check client, server, outbound handlers int getMaxIdleTime() { return JiveGlobals.getIntProperty( "xmpp.server.idle", 6 * 60 * 1000 ) / 1000; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java index 5abfc84ffb..ab7980ec99 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -1,3 +1,19 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.nio; import io.netty.bootstrap.Bootstrap; @@ -27,10 +43,12 @@ import static org.jivesoftware.openfire.session.Session.Log; +/** + * Initialises an outgoing netty channel for outbound S2S + */ public class NettySessionInitializer { private static final Logger LOG = LoggerFactory.getLogger(NettySessionInitializer.class); - private final DomainPair domainPair; private final int port; private boolean directTLS = false; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java index de05a41f9f..0c854404ed 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyXMPPDecoder.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,16 +25,18 @@ import java.util.List; /** - * Decoder class that parses ByteBuffers and generates XML stanzas. Generated + * Decoder that parses ByteBuffers and generates XML stanzas. Generated * stanzas are then passed to the next filters. */ public class NettyXMPPDecoder extends ByteToMessageDecoder { @Override protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { - // Get the XML light parser from the channel + // Get the XML parser from the channel XMLLightweightParser parser = ctx.channel().attr(NettyConnectionHandler.XML_PARSER).get(); + // Check that the buffer is not bigger than 1 Megabyte. For security reasons + // we will abort parsing when 1 Mega of queued chars was found. if (parser.isMaxBufferSizeExceeded()) { in.release(); return; @@ -44,6 +46,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) t char[] readChars = in.readCharSequence(in.readableBytes(), CharsetUtil.UTF_8).toString().toCharArray(); parser.read(readChars); + // Add any decoded messages to our outbound list to be processed by subsequent channelRead() events if (parser.areThereMsgs()) { out.addAll(Arrays.asList(parser.getMsgs())); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java index cdcd3a7e75..ccedce50ed 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java @@ -1,3 +1,19 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.nio; import org.apache.mina.core.session.IoSession; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java index 208a555fc4..6d9a9e8d06 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMLLightweightParser.java @@ -38,9 +38,9 @@ class XMLLightweightParser { private static final String MAX_PROPERTY_NAME = "xmpp.parser.buffer.size"; private static int maxBufferSize; - // Chars that rappresent CDATA section start + // Chars that represent CDATA section start protected static char[] CDATA_START = {'<', '!', '[', 'C', 'D', 'A', 'T', 'A', '['}; - // Chars that rappresent CDATA section end + // Chars that represent CDATA section end protected static char[] CDATA_END = {']', ']', '>'}; // Buffer with all data retrieved @@ -82,7 +82,7 @@ class XMLLightweightParser { protected int startLastMsg = 0; // Flag used to discover tag in the form . protected boolean insideRootTag = false; - // Object conteining the head tag + // Object containing the head tag protected StringBuilder head = new StringBuilder(16); // List with all finished messages found. protected List msgs = new ArrayList<>(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java index d02f6a8abe..c512f1d51e 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. + * Copyright (C) 2005-2008 Jive Software, 2023 Ignite Realtime Foundation. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java index fd017d7468..b76e2603a7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2005-2008 Jive Software, 2015-2021 Ignite Realtime Foundation. All rights reserved. + * Copyright (C) 2005-2008 Jive Software, 2015-2023 Ignite Realtime Foundation. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -45,9 +45,9 @@ import java.util.concurrent.locks.Lock; /** - * An OutgoingSessionPromise provides an asynchronic way for sending packets to remote servers. + * An OutgoingSessionPromise provides an asynchronous way for sending packets to remote servers. * When looking for a route to a remote server that does not have an existing connection, a session - * promise is returned. + * promise is returned.

* * This class will queue packets and process them in another thread. The processing thread will * use a pool of thread that will actually do the hard work. The threads in the pool will try @@ -286,7 +286,7 @@ private RoutableChannelHandler establishConnection() throws Exception { final Lock lock = serversCache.getLock(domainPair); lock.lock(); try { - created = LocalOutgoingServerSession.authenticateDomain(domainPair); // TODO: AG MAGIC HERE + created = LocalOutgoingServerSession.authenticateDomain(domainPair); } finally { lock.unlock(); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java index 88f3f20cb3..613291fb56 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java @@ -41,7 +41,7 @@ // TODO most getters in this class assume that the ConnectionAcceptor property value match the property values of JiveGlobals. This should be the case, but should be asserted. public class ConnectionListener { - private Logger Log; + private final Logger Log; // Connection characteristics private final ConnectionType type; @@ -199,7 +199,7 @@ public synchronized void start() { case BOSH_C2S: case WEBADMIN: - Log.debug( "Not starting a (MINA-based) connection acceptor, as connections of type " + getType() + " depend on another IO technology."); + Log.debug( "Not starting a (Netty-based) connection acceptor, as connections of type " + getType() + " depend on another IO technology."); return; default: @@ -229,7 +229,6 @@ public synchronized void start() Log.debug( "Starting..." ); if ( getType() == ConnectionType.SOCKET_S2S || getType() == ConnectionType.SOCKET_C2S ) { connectionAcceptor = new NettyConnectionAcceptor( generateConnectionConfiguration() ); -// connectionAcceptor = new LegacyConnectionAcceptor( generateConnectionConfiguration() ); } else { connectionAcceptor = new MINAConnectionAcceptor( generateConnectionConfiguration() ); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 8d39cdaf74..3d0b6ba5f3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -1,3 +1,19 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.spi; import io.netty.bootstrap.ServerBootstrap; @@ -17,8 +33,7 @@ import java.net.InetSocketAddress; /** - * This class is responsible for accepting new (socket) connections, using Java NIO implementation provided by the - * Netty framework. + * Responsible for accepting new (socket) connections, using Java NIO implementation provided by the Netty framework. * * @author Matthew Vivian * @author Alex Gidman @@ -68,25 +83,16 @@ public NettyConnectionAcceptor(ConnectionConfiguration configuration) { default: throw new IllegalStateException("This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType()); } -// connectionHandler = new NettyServerConnectionHandler( configuration ); -// switch ( configuration.getType() ) -// { -// case SOCKET_S2S: -// connectionHandler = new ServerConnectionHandler( configuration ); -// break; -// case SOCKET_C2S: -// connectionHandler = new ClientConnectionHandler( configuration ); -// break; + +// TODO add support for COMPONENT & Multiplexer // case COMPONENT: // connectionHandler = new ComponentConnectionHandler( configuration ); // break; // case CONNECTION_MANAGER: // connectionHandler = new MultiplexerConnectionHandler( configuration ); // break; -// default: -// throw new IllegalStateException( "This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType() ); -// } + } /** From cc177aa292610bc7352cccc1af70a5094460c455 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 17 Jul 2023 14:30:09 +0100 Subject: [PATCH 16/56] OF-2559 Refactor: extracted isDirectTLS method --- .../openfire/spi/NettyConnectionAcceptor.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 3d0b6ba5f3..9b7259e0a8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -70,7 +70,7 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { public NettyConnectionAcceptor(ConnectionConfiguration configuration) { super(configuration); - String name = configuration.getType().toString().toLowerCase() + (configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode ? "_ssl" : ""); + String name = configuration.getType().toString().toLowerCase() + (isDirectTLS() ? "_ssl" : ""); Log = LoggerFactory.getLogger( NettyConnectionAcceptor.class.getName() + "[" + name + "]" ); switch (configuration.getType()) { @@ -110,7 +110,7 @@ public synchronized void start() { // Instantiate a new Channel to accept incoming connections. .channel(NioServerSocketChannel.class) // The handler specified here will always be evaluated by a newly accepted Channel. - .childHandler(new NettyServerInitializer(connectionHandler, configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode)) + .childHandler(new NettyServerInitializer(connectionHandler, isDirectTLS())) // Set the listen backlog (queue) length. .option(ChannelOption.SO_BACKLOG, JiveGlobals.getIntProperty("xmpp.socket.backlog", 50)) // option() is for the NioServerSocketChannel that accepts incoming connections. @@ -152,6 +152,10 @@ public synchronized void start() { } } + private boolean isDirectTLS() { + return configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode; + } + /** * Stops this acceptor by unbinding the socket acceptor. Does nothing when the instance is not started. */ From fe819fb3d5af1bf0fa12fb0bb1a492271f4a4ac5 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 17 Jul 2023 14:42:04 +0100 Subject: [PATCH 17/56] OF-2559 Refactor: Log to application log --- .../org/jivesoftware/openfire/nio/NettyConnection.java | 2 +- .../openfire/nio/NettyConnectionHandler.java | 9 +++------ .../openfire/nio/NettyServerConnectionHandler.java | 7 +++++-- .../openfire/nio/NettySessionInitializer.java | 2 +- .../openfire/spi/NettyConnectionAcceptor.java | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 33adb28bde..a696ed1fd7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -346,7 +346,7 @@ public void deliver(Packet packet) throws UnauthorizedException { @Override public void deliverRawText(String text) { - System.out.println("Sending: " + text); + Log.trace("Sending: " + text); if (!isClosed()) { boolean errorDelivering = false; ChannelFuture f = channelHandlerContext.writeAndFlush(text); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index 551f21bb08..6734725ed4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -99,7 +99,7 @@ protected NettyConnectionHandler(ConnectionConfiguration configuration ) { @Override public void handlerAdded(ChannelHandlerContext ctx) { - System.out.println("Netty XMPP handler added: " + ctx.channel().localAddress()); + Log.trace("Netty XMPP handler added: " + ctx.channel().localAddress()); // ConnectionHandler.sessionOpened() // TODO - do we want a separate parser per-channel? @@ -125,7 +125,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { @Override public void handlerRemoved(ChannelHandlerContext ctx) { - System.out.println("Netty XMPP handler removed: " + ctx.channel().localAddress()); + Log.trace("Netty XMPP handler removed: " + ctx.channel().localAddress()); } @Override @@ -139,8 +139,7 @@ public void channelRead0(ChannelHandlerContext ctx, String message) { // Update counter of read bytes // updateReadBytesCounter(session); TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- - - System.out.println("Received: " + message); + Log.trace("Handler on " + ctx.channel().localAddress() + " received: " + message); // Let the stanza handler process the received stanza try { ctx.channel().attr(HANDLER).get().process(message, parser); @@ -161,8 +160,6 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { } - - /** * Updates the system counter of read bytes. This information is used by the incoming * bytes statistic. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java index 87f7f616e3..fbeed29043 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -25,6 +25,8 @@ import org.jivesoftware.openfire.spi.ConnectionConfiguration; import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.SystemProperty; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Server-specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created @@ -35,6 +37,8 @@ */ public class NettyServerConnectionHandler extends NettyConnectionHandler { + private static final Logger Log = LoggerFactory.getLogger(NettyServerConnectionHandler.class); + /** * Enable / disable backup delivery of stanzas to the XMPP server itself when a stanza failed to be delivered on a * server-to-server connection. When disabled, stanzas that can not be delivered on the connection are discarded. @@ -65,11 +69,10 @@ StanzaHandler createStanzaHandler(NettyConnection connection) { @Override public void handlerAdded(ChannelHandlerContext ctx) { - System.out.println("Adding NettyServerConnectionHandler"); + Log.trace("Adding NettyServerConnectionHandler"); super.handlerAdded(ctx); } - // TODO - check idle time is set for connections (check client, server, outbound handlers int getMaxIdleTime() { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java index ab7980ec99..9e1dc34009 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -148,7 +148,7 @@ private void sendOpeningStreamHeader(Channel channel) { sb.append(" to=\"").append(domainPair.getRemote()).append("\""); sb.append(" version=\"1.0\">"); channel.writeAndFlush(sb.toString()); - System.out.println("Sending: " + sb.toString()); + Log.trace("Sending: " + sb); } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 9b7259e0a8..609251cf57 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -101,7 +101,7 @@ public NettyConnectionAcceptor(ConnectionConfiguration configuration) { */ @Override public synchronized void start() { - System.out.println("Running Netty on port: " + getPort()); + Log.debug("Running Netty on port: " + getPort()); try { // ServerBootstrap is a helper class that sets up a server From e71831c14f5415edd9f8a2c37d3f28a73a38b04f Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 18 Jul 2023 11:16:27 +0100 Subject: [PATCH 18/56] OF-2559 Deprecate Mina-based ClientConnectionHandler We are moving to Netty. --- .../org/jivesoftware/openfire/nio/ClientConnectionHandler.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java index 4c01454cde..a337217c01 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java @@ -40,7 +40,9 @@ * be created and how to build and configure a {@link NIOConnection}. * * @author Gaston Dombiak + * @deprecated Use {@link NettyClientConnectionHandler} instead */ +@Deprecated public class ClientConnectionHandler extends ConnectionHandler { private static final Logger Log = LoggerFactory.getLogger(ClientConnectionHandler.class); From ad3c115a217db1d3b59cef77045363ef6f9fe1ca Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 18 Jul 2023 11:18:01 +0100 Subject: [PATCH 19/56] OF-2559 Improved comments --- .../net/RespondingServerStanzaHandler.java | 3 +- .../openfire/net/ServerStanzaHandler.java | 4 +-- .../openfire/nio/NettyConnectionHandler.java | 2 -- .../nio/NettyServerConnectionHandler.java | 2 +- .../openfire/nio/NettySessionInitializer.java | 3 +- .../spi/EncryptionArtifactFactory.java | 29 +++++++++++++++++++ 6 files changed, 35 insertions(+), 8 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index 8146deefd3..8ae99fe83e 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -46,7 +46,6 @@ public class RespondingServerStanzaHandler extends StanzaHandler { private static final Logger LOG = LoggerFactory.getLogger(RespondingServerStanzaHandler.class); private final DomainPair domainPair; - private boolean isSessionAuthenticated = false; /** @@ -118,7 +117,7 @@ private static boolean isRelevantNamespace(Namespace ns) { } @Override - boolean processUnknowPacket(Element doc) throws UnauthorizedException { + boolean processUnknowPacket(Element doc) { String rootTagName = doc.getName(); // Handle features diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java index 7d74d1b494..6ab3ef72a4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ServerStanzaHandler.java @@ -45,9 +45,9 @@ * The connection used for receiving packets will use a ServerStanzaHandler since the other * connection will not receive packets.

* - * TODO Finish migration of s2s to use NIO instead of blocking threads. Migrate from ServerSocketReader. - * * @author Gaston Dombiak + * @author Alex Gidman + * @author Matthew Vivian */ public class ServerStanzaHandler extends StanzaHandler { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index 6734725ed4..360094e9eb 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -100,9 +100,7 @@ protected NettyConnectionHandler(ConnectionConfiguration configuration ) { @Override public void handlerAdded(ChannelHandlerContext ctx) { Log.trace("Netty XMPP handler added: " + ctx.channel().localAddress()); - // ConnectionHandler.sessionOpened() - // TODO - do we want a separate parser per-channel? // Create a new XML parser for the new connection. The parser will be used by the XMPPDecoder filter. ctx.channel().attr(XML_PARSER).set(new XMLLightweightParser()); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java index fbeed29043..eb6bc00e85 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -44,7 +44,7 @@ public class NettyServerConnectionHandler extends NettyConnectionHandler * server-to-server connection. When disabled, stanzas that can not be delivered on the connection are discarded. */ public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.server.netty-backup-packet-delivery.enabled") // TODO - rename once MINA-specific is removed and NettyServerConnectionHandler becomes ServerConnectionHandler + .setKey("xmpp.server.netty-backup-packet-delivery.enabled") .setDefaultValue(true) .setDynamic(true) .build(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java index 9e1dc34009..032b9235b1 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -116,8 +116,9 @@ public boolean exceptionOccurredForDirectTLS(Throwable cause) { return waitForSession(channel); } catch (InterruptedException e) { + Log.error("Error establishing Netty client session", e); stop(); - throw new RuntimeException(e); // TODO: Better to throw all exceptions and catch outside? + throw new RuntimeException(e); } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index fd04677537..dc585cb2a7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -1,3 +1,19 @@ +/* + * Copyright (C) 2015-2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.jivesoftware.openfire.spi; import io.netty.handler.ssl.ClientAuth; @@ -327,6 +343,12 @@ public synchronized SslContextFactory getSslContextFactory() } } + /** + * Create and configure a new SslContext instance for a Netty server.

+ * + * @param directTLS if the first write request should be encrypted. + * @return A secure socket protocol implementation which acts as a factory for {@link SSLContext} and {@link io.netty.handler.ssl.SslHandler} + */ public SslContext createServerModeSslContext(boolean directTLS) throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { getKeyManagers(); SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); @@ -352,6 +374,13 @@ public SslContext createServerModeSslContext(boolean directTLS) throws Unrecover return builder.build(); } + /** + * Create and configure a new SslContext instance for a Netty client.

+ * + * Used when the Openfire server is acting as a client when making S2S connections. + * + * @return A secure socket protocol implementation which acts as a factory for {@link SSLContext} and {@link io.netty.handler.ssl.SslHandler} + */ public SslContext createClientModeSslContext() throws SSLException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException { getKeyManagers(); From 4ee6a3971f8ab7ea6f0a2b53d8beabdedb9ff481 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 18 Jul 2023 11:19:34 +0100 Subject: [PATCH 20/56] OF-2559 Removed unused connection handler Was part of an incomplete migration to Mina. --- .../openfire/nio/ServerConnectionHandler.java | 72 ------------------- 1 file changed, 72 deletions(-) delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java deleted file mode 100644 index ccedce50ed..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.session.IoSession; -import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.PacketDeliverer; -import org.jivesoftware.openfire.XMPPServer; -import org.jivesoftware.openfire.net.ServerStanzaHandler; -import org.jivesoftware.openfire.net.StanzaHandler; -import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.util.JiveGlobals; -import org.jivesoftware.util.SystemProperty; - -/** - * ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created and how to build and configure - * a {@link NIOConnection}. - * - * @author Guus der Kinderen, guus.der.kinderen@gmail.com - */ -public class ServerConnectionHandler extends ConnectionHandler -{ - /** - * Enable / disable backup delivery of stanzas to the XMPP server itself when a stanza failed to be delivered on a - * server-to-server connection. When disabled, stanzas that can not be delivered on the connection are discarded. - */ - public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.server.backup-packet-delivery.enabled") - .setDefaultValue(true) - .setDynamic(true) - .build(); - private final boolean directTLS; - - public ServerConnectionHandler( ConnectionConfiguration configuration ) - { - super( configuration ); - this.directTLS = configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode; - } - - @Override - NIOConnection createNIOConnection( IoSession session ) - { - final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? XMPPServer.getInstance().getPacketDeliverer() : null; - return new NIOConnection(session, backupDeliverer, configuration); - } - - @Override - StanzaHandler createStanzaHandler( NIOConnection connection ) - { - return new ServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection, directTLS ); - } - - @Override - int getMaxIdleTime() - { - return JiveGlobals.getIntProperty( "xmpp.server.idle", 6 * 60 * 1000 ) / 1000; - } -} From 04ecba7ded7bb8f6ef36d86c9a799d72ca0fb287 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 18 Jul 2023 11:21:19 +0100 Subject: [PATCH 21/56] OF-2559 Netty is always used for S2S and C2S Explicitly show this by removing the Mina-based implementations from the ConnectionAcceptor. --- .../jivesoftware/openfire/spi/MINAConnectionAcceptor.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java index 0719d8bede..c351cf001f 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java @@ -60,12 +60,6 @@ public MINAConnectionAcceptor( ConnectionConfiguration configuration ) switch ( configuration.getType() ) { - case SOCKET_S2S: - connectionHandler = new ServerConnectionHandler( configuration ); - break; - case SOCKET_C2S: - connectionHandler = new ClientConnectionHandler( configuration ); - break; case COMPONENT: connectionHandler = new ComponentConnectionHandler( configuration ); break; From 31c1b239a9d99dc74fbe4dddd7c765940b9c295d Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 18 Jul 2023 11:22:01 +0100 Subject: [PATCH 22/56] OF-2559 Refactor to remove unused code following Netty migration --- .../session/LocalOutgoingServerSession.java | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index ca4085e366..7b2f2fef71 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -18,32 +18,20 @@ import com.google.common.collect.Interner; import com.google.common.collect.Interners; -import org.dom4j.DocumentException; -import org.dom4j.Element; -import org.dom4j.io.XMPPPacketReader; import org.jivesoftware.openfire.*; import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.event.ServerSessionEventDispatcher; -import org.jivesoftware.openfire.net.MXParser; -import org.jivesoftware.openfire.net.SASLAuthentication; -import org.jivesoftware.openfire.net.SocketConnection; import org.jivesoftware.openfire.net.SocketUtil; import org.jivesoftware.openfire.nio.NettySessionInitializer; import org.jivesoftware.openfire.server.OutgoingServerSocketReader; import org.jivesoftware.openfire.server.RemoteServerManager; import org.jivesoftware.openfire.server.ServerDialback; -import org.jivesoftware.openfire.spi.BasicStreamIDFactory; -import org.jivesoftware.util.StringUtils; import org.jivesoftware.util.TaskEngine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.xmlpull.v1.XmlPullParser; -import org.xmlpull.v1.XmlPullParserException; import org.xmpp.packet.*; import javax.annotation.Nonnull; -import java.io.IOException; -import java.io.InputStreamReader; import java.net.Socket; import java.net.SocketAddress; import java.nio.charset.StandardCharsets; @@ -51,7 +39,6 @@ import java.security.cert.CertificateException; import java.util.Collection; import java.util.HashSet; -import java.util.Iterator; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; From e186bc52c6a830b2920920a6240a76a1758f2a9a Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Tue, 18 Jul 2023 13:50:51 +0100 Subject: [PATCH 23/56] feat: OF-2559 - surface session initialise timeout to a system property --- .../main/resources/openfire_i18n.properties | 1 + .../session/LocalOutgoingServerSession.java | 23 +++++++++++++++---- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/i18n/src/main/resources/openfire_i18n.properties b/i18n/src/main/resources/openfire_i18n.properties index c8fb7cfd1c..5c3c819d00 100644 --- a/i18n/src/main/resources/openfire_i18n.properties +++ b/i18n/src/main/resources/openfire_i18n.properties @@ -1683,6 +1683,7 @@ system_property.xmpp.server.outgoing.max.threads=Minimum amount of threads in th system_property.xmpp.server.outgoing.min.threads=Maximum amount of threads in the thread pool that is used to establish outbound server-to-server connections system_property.xmpp.server.outgoing.threads-timeout=Amount of time after which idle, surplus threads are removed from the thread pool that is used to establish outbound server-to-server connections. system_property.xmpp.server.outgoing.queue=Maximum amount of outbound server-to-server connections that can be in process of establishment in the thread pool (surplus connections will be created on the calling thread, possibly / gracefully slowing down other operations considerably) +system_property.xmpp.server.session.initialise-timeout=Maximum amount of time in seconds for an outbound S2S session to be initialised system_property.cluster-monitor.service-enabled=Set to true to send messages to admins on cluster events, otherwise false system_property.ldap.override.avatar=Set to true to save avatars in the local database, otherwise false system_property.xmpp.domain=The XMPP domain of this server. Do not change this property directly, instead re-run the setup process. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index 7b2f2fef71..9809324cd5 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -26,6 +26,7 @@ import org.jivesoftware.openfire.server.OutgoingServerSocketReader; import org.jivesoftware.openfire.server.RemoteServerManager; import org.jivesoftware.openfire.server.ServerDialback; +import org.jivesoftware.util.SystemProperty; import org.jivesoftware.util.TaskEngine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +35,8 @@ import javax.annotation.Nonnull; import java.net.Socket; import java.net.SocketAddress; +import java.time.Duration; +import java.time.temporal.ChronoUnit; import java.nio.charset.StandardCharsets; import java.security.cert.CertPathValidatorException; import java.security.cert.CertificateException; @@ -70,6 +73,16 @@ public class LocalOutgoingServerSession extends LocalServerSession implements Ou private static final Interner remoteAuthMutex = Interners.newWeakInterner(); + /** + * Controls the S2S outgoing session initialise timeout time in seconds + */ + public static final SystemProperty INITIALISE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Duration.class) + .setKey("xmpp.server.session.initialise-timeout") + .setDefaultValue(Duration.ofSeconds(5)) + .setChronoUnit(ChronoUnit.SECONDS) + .setDynamic(true) + .build(); + private OutgoingServerSocketReader socketReader; private final Collection outgoingDomainPairs = new HashSet<>(); @@ -444,9 +457,9 @@ private static LocalOutgoingServerSession attemptDialbackOverTLS(Connection conn else { log.debug("Skipping server dialback attempt as it has been disabled by local configuration."); return null; - } + } } - + private static LocalOutgoingServerSession attemptSASLexternal(SocketConnection connection, MXParser xpp, XMPPPacketReader reader, DomainPair domainPair, String id, StringBuilder openingStream) throws DocumentException, IOException, XmlPullParserException { final Logger log = LoggerFactory.getLogger( Log.getName() + "[EXTERNAL SASL for: " + domainPair + " (Stream ID: " + id + ")]" ); @@ -455,7 +468,7 @@ private static LocalOutgoingServerSession attemptSASLexternal(SocketConnection c log.debug("EXTERNAL SASL was successful."); // SASL was successful so initiate a new stream connection.deliverRawText(openingStream.toString()); - + // Reset the parser //xpp.resetInput(); // // Reset the parser to use the new secured reader @@ -479,9 +492,9 @@ private static LocalOutgoingServerSession attemptSASLexternal(SocketConnection c else { log.debug("EXTERNAL SASL failed."); return null; - } + } } - + private static boolean doExternalAuthentication(String localDomain, SocketConnection connection, XMPPPacketReader reader) throws DocumentException, IOException, XmlPullParserException { From 30237c627240c7baf6b98dea3fe01a3265be2a5f Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Thu, 20 Jul 2023 12:16:38 +0100 Subject: [PATCH 24/56] feat: OF-2559 - implement handling for idle states in inbound C2S and S2S connections Adds IdleStateHandler and NettyIdleStateKeepAliveHandler to the NettyServerInitializer pipeline code. If an inbound session idles Openfire will either send a ping to keep the connection alive, or close the connection. --- .../main/resources/openfire_i18n.properties | 3 +- .../nio/NettyClientConnectionHandler.java | 34 +---- .../openfire/nio/NettyConnectionHandler.java | 12 +- .../nio/NettyIdleStateKeepAliveHandler.java | 122 ++++++++++++++++++ .../nio/NettyOutboundConnectionHandler.java | 6 +- .../nio/NettyServerConnectionHandler.java | 22 +++- .../openfire/session/ConnectionSettings.java | 1 + .../openfire/spi/NettyServerInitializer.java | 19 ++- 8 files changed, 166 insertions(+), 53 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java diff --git a/i18n/src/main/resources/openfire_i18n.properties b/i18n/src/main/resources/openfire_i18n.properties index 5c3c819d00..15546d7e86 100644 --- a/i18n/src/main/resources/openfire_i18n.properties +++ b/i18n/src/main/resources/openfire_i18n.properties @@ -1671,7 +1671,7 @@ system_property.sasl.scram-sha-1.iteration-count=The number of iterations when s system_property.xmpp.auth.anonymous=Set to true to allow anonymous login, otherwise false system_property.xmpp.auth.external.client.skip-cert-revalidation=Set to true to avoid validation of the client-provided PKIX certificate (for mutual authentication) other than the validation that happens when the TLS session is established. system_property.xmpp.auth.ssl.default-trustmanager-impl=The class to use as the default SSL/TLS TrustManager (which checks certificates from peers). -system_property.xmpp.client.idle=How long, in milliseconds, before idle sessions are dropped. Set to -1 to never drop idle sessions. +system_property.xmpp.client.idle=How long, in milliseconds, before idle client sessions are dropped. Set to -1 to never drop idle sessions. system_property.xmpp.client.idle.ping=Set to true to ping idle clients, otherwise false system_property.xmpp.client.version-query.enabled=Send a version request query to clients when they connect. system_property.xmpp.client.version-query.delay=After this amount of time has passed since a new client connection has been accepted, a version request is being sent to the peer. @@ -1684,6 +1684,7 @@ system_property.xmpp.server.outgoing.min.threads=Maximum amount of threads in th system_property.xmpp.server.outgoing.threads-timeout=Amount of time after which idle, surplus threads are removed from the thread pool that is used to establish outbound server-to-server connections. system_property.xmpp.server.outgoing.queue=Maximum amount of outbound server-to-server connections that can be in process of establishment in the thread pool (surplus connections will be created on the calling thread, possibly / gracefully slowing down other operations considerably) system_property.xmpp.server.session.initialise-timeout=Maximum amount of time in seconds for an outbound S2S session to be initialised +system_property.xmpp.server.idle=How long, in milliseconds, before idle inbound server sessions are dropped. Set to -1 to never drop idle sessions. system_property.cluster-monitor.service-enabled=Set to true to send messages to admins on cluster events, otherwise false system_property.ldap.override.avatar=Set to true to save avatars in the local database, otherwise false system_property.xmpp.domain=The XMPP domain of this server. Do not change this property directly, instead re-run the setup process. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java index ea49b67d0e..20e1927ef8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java @@ -17,14 +17,12 @@ package org.jivesoftware.openfire.nio; import io.netty.channel.ChannelHandlerContext; -import org.apache.mina.core.session.IdleStatus; -import org.apache.mina.core.session.IoSession; import org.jivesoftware.openfire.PacketDeliverer; import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.ClientStanzaHandler; import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.session.ConnectionSettings; import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.SystemProperty; /** @@ -60,37 +58,13 @@ NettyConnection createNettyConnection(ChannelHandlerContext ctx) { @Override StanzaHandler createStanzaHandler(NettyConnection connection) { - return new ClientStanzaHandler(XMPPServer.getInstance().getPacketRouter(), connection); + return new ClientStanzaHandler(XMPPServer.getInstance ().getPacketRouter(), connection); } - - // TODO Do we need a sessionIdle function for the ClientConnectionHandler specifically? : - /** - * In addition to the functionality provided by the parent class, this - * method will send XMPP ping requests to the remote entity on every first - * invocation of this method (which will occur after a period of half the - * allowed connection idle time has passed, without any IO). - * - * XMPP entities must respond with either an IQ result or an IQ error - * (feature-unavailable) stanza upon receiving the XMPP ping stanza. Both - * responses will be received by Openfire and will cause the connection idle - * count to be reset. - * - * Entities that do not respond to the IQ Ping stanzas can be considered - * dead, and their connection will be closed by the parent class - * implementation on the second invocation of this method. - * - * Note that whitespace pings that are sent by XMPP entities will also cause - * the connection idle count to be reset. - * - * @see ConnectionHandler#sessionIdle(IoSession, IdleStatus) - */ - @Override - int getMaxIdleTime() - { - return JiveGlobals.getIntProperty( "xmpp.server.idle", 6 * 60 * 1000 ) / 1000; + public int getMaxIdleTime() { + return (int) ConnectionSettings.Client.IDLE_TIMEOUT_PROPERTY.getValue().toSeconds(); } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index 360094e9eb..c22bc12b83 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -95,7 +95,7 @@ protected NettyConnectionHandler(ConnectionConfiguration configuration ) { * * @return the max number of seconds a connection can be idle. */ - abstract int getMaxIdleTime(); + public abstract int getMaxIdleTime(); @Override public void handlerAdded(ChannelHandlerContext ctx) { @@ -109,16 +109,6 @@ public void handlerAdded(ChannelHandlerContext ctx) { ctx.channel().attr(CONNECTION).set(nettyConnection); ctx.channel().attr(HANDLER).set(createStanzaHandler(nettyConnection)); - - // Set the max time a connection can be idle before closing it. This amount of seconds - // is divided in two, as Openfire will ping idle clients first (at 50% of the max idle time) - // before disconnecting them (at 100% of the max idle time). This prevents Openfire from - // removing connections without warning. -// TODO idle handler see: https://netty.io/4.0/api/io/netty/handler/timeout/IdleStateHandler.html -// final int idleTime = getMaxIdleTime() / 2; -// if (idleTime > 0) { -// session.getConfig().setIdleTime(IdleStatus.READER_IDLE, idleTime); -// } } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java new file mode 100644 index 0000000000..7576497adb --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java @@ -0,0 +1,122 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.jivesoftware.openfire.nio; + +import io.netty.channel.ChannelDuplexHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; +import org.jivesoftware.openfire.SessionManager; +import org.jivesoftware.openfire.XMPPServer; +import org.jivesoftware.openfire.auth.UnauthorizedException; +import org.jivesoftware.openfire.handler.IQPingHandler; +import org.jivesoftware.openfire.session.ConnectionSettings; +import org.jivesoftware.openfire.session.LocalClientSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xmpp.packet.IQ; +import org.xmpp.packet.JID; + +import static org.jivesoftware.openfire.nio.NettyConnectionHandler.CONNECTION; + +/** + * A NettyIdleStateKeepAliveHandler listens for IdleStateEvents triggered by an + * IdleStateHandler. The readerIdleTimeSeconds in IdleStateHandler should be set to + * a value higher than writerIdleTimeSeconds (typically double), else the connection + * will always be closed. + * + * XMPP entities must respond with either an IQ result or an IQ error + * (feature-unavailable) stanza upon receiving the XMPP ping stanza. Both + * responses will be received by Openfire and will cause the connection idle + * count to be reset. + * + * Entities that do not respond to the IQ Ping stanzas can be considered + * dead, and their connection will be closed when the IdleStateHandler triggers an idle + * read state. + * + * Note that whitespace pings that are sent by XMPP entities will also cause + * the connection idle count to be reset. + * + * @see IdleStateEvent + * @see io.netty.handler.timeout.IdleStateHandler + * + * @author Alex Gidman + * */ +public class NettyIdleStateKeepAliveHandler extends ChannelDuplexHandler { + + private final boolean clientConnection; + private static final Logger Log = LoggerFactory.getLogger(NettyIdleStateKeepAliveHandler.class); + + public NettyIdleStateKeepAliveHandler(boolean clientConnection) { + this.clientConnection = clientConnection; + } + + /** + * Processes IdleStateEvents triggered by an IdleStateHandler. + * If the IdleStateEvent is an idle read state, the Netty channel is closed. + * If the IdleStateEvent is an idle write state, an XMPP ping request is sent + * to the remote entity. + * + * @param ctx ChannelHandlerContext + * @param evt Event caught, expect IdleStateEvent + * @throws Exception when attempting to deliver ping packet + */ + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + IdleStateEvent e = (IdleStateEvent) evt; + final boolean doPing = ConnectionSettings.Client.KEEP_ALIVE_PING_PROPERTY.getValue() && clientConnection; + if (e.state() == IdleState.READER_IDLE) { + ctx.close(); + } else if (doPing && e.state() == IdleState.WRITER_IDLE) { + sendPingPacket(ctx); + } + } + } + + /** + * Sends an IQ ping packet on the channel associated with the channel handler context. + * + * @param ctx ChannelHandlerContext + * @throws UnauthorizedException when attempting to deliver ping packet + */ + private void sendPingPacket(ChannelHandlerContext ctx) throws UnauthorizedException { + NettyConnection connection = ctx.channel().attr(CONNECTION).get(); + JID entity = connection.session.getAddress(); + if (entity != null) { + // Ping the connection to see if it is alive. + final IQ pingRequest = new IQ(IQ.Type.get); + pingRequest.setChildElement("ping", IQPingHandler.NAMESPACE); + pingRequest.setFrom(XMPPServer.getInstance().getServerInfo().getXMPPDomain()); + pingRequest.setTo(entity); + + if (Log.isDebugEnabled()) { + Log.debug("Pinging connection that has been idle: " + connection); + } + + // OF-1497: Ensure that data sent to the client is processed through LocalClientSession, to avoid + // synchronisation issues with stanza counts related to Stream Management (XEP-0198)! + LocalClientSession ofSession = (LocalClientSession) SessionManager.getInstance().getSession( entity ); + if (ofSession == null) { + Log.warn( "Trying to ping a Netty connection that's idle, but has no corresponding Openfire session. Netty Connection: " + connection ); + } else { + ofSession.deliver( pingRequest ); + } + } + } + +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java index 6d8ec93ef3..e568616669 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -20,8 +20,10 @@ import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.RespondingServerStanzaHandler; import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.session.ConnectionSettings; import org.jivesoftware.openfire.session.DomainPair; import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.jivesoftware.util.JiveGlobals; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,8 +54,8 @@ StanzaHandler createStanzaHandler(NettyConnection connection) { } @Override - int getMaxIdleTime() { - return 0; + public int getMaxIdleTime() { + return JiveGlobals.getIntProperty(ConnectionSettings.Server.IDLE_TIMEOUT_PROPERTY, 360); } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java index eb6bc00e85..bae6a368d8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -23,11 +23,13 @@ import org.jivesoftware.openfire.net.ServerStanzaHandler; import org.jivesoftware.openfire.net.StanzaHandler; import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.SystemProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; +import java.time.temporal.ChronoUnit; + /** * Server-specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created * and how to build and configure a {@link NettyConnection}. @@ -39,6 +41,18 @@ public class NettyServerConnectionHandler extends NettyConnectionHandler { private static final Logger Log = LoggerFactory.getLogger(NettyServerConnectionHandler.class); + /** + * How much time the server will wait without receiving an inbound message until it is classed as an idle connection + * and closed. + */ + public static final SystemProperty IDLE_TIMEOUT_PROPERTY = SystemProperty.Builder.ofType(Duration.class) + .setKey("xmpp.server.idle") + .setDefaultValue(Duration.ofMinutes(6)) + .setMinValue(Duration.ofMillis(-1)) + .setChronoUnit(ChronoUnit.MILLIS) + .setDynamic(true) + .build(); + /** * Enable / disable backup delivery of stanzas to the XMPP server itself when a stanza failed to be delivered on a * server-to-server connection. When disabled, stanzas that can not be delivered on the connection are discarded. @@ -73,9 +87,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { super.handlerAdded(ctx); } - // TODO - check idle time is set for connections (check client, server, outbound handlers - int getMaxIdleTime() - { - return JiveGlobals.getIntProperty( "xmpp.server.idle", 6 * 60 * 1000 ) / 1000; + public int getMaxIdleTime() { + return Math.toIntExact(IDLE_TIMEOUT_PROPERTY.getValue().toSeconds()); } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java index d731a8a856..80c08eafcb 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/ConnectionSettings.java @@ -100,6 +100,7 @@ public static final class Server { public static final String PERMISSION_SETTINGS = "xmpp.server.permission"; public static final String AUTH_PER_CLIENTCERT_POLICY = "xmpp.server.cert.policy"; public static final String ALLOW_ANONYMOUS_OUTBOUND_DATA = "xmpp.server.allow-anonymous-outbound-data"; + public static final String IDLE_TIMEOUT_PROPERTY = "xmpp.server.idle"; public static final String STRICT_CERTIFICATE_VALIDATION = "xmpp.socket.ssl.certificate.strict-validation"; } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java index f0f13d081a..d0f9dd09ef 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java @@ -4,8 +4,11 @@ import io.netty.channel.ChannelPipeline; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.string.StringEncoder; +import io.netty.handler.timeout.IdleStateHandler; import io.netty.handler.timeout.WriteTimeoutHandler; +import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; import org.jivesoftware.openfire.nio.NettyConnectionHandler; +import org.jivesoftware.openfire.nio.NettyIdleStateKeepAliveHandler; import org.jivesoftware.openfire.nio.NettyXMPPDecoder; import org.jivesoftware.util.SystemProperty; @@ -35,14 +38,22 @@ public class NettyServerInitializer extends ChannelInitializer { public NettyServerInitializer(NettyConnectionHandler businessLogicHandler, boolean directTLS) { this.businessLogicHandler = businessLogicHandler; this.directTLS = directTLS; + } @Override public void initChannel(SocketChannel ch) throws Exception { - ch.pipeline().addLast(new NettyXMPPDecoder()); - ch.pipeline().addLast(new StringEncoder()); - ch.pipeline().addLast("stalledSessionHandler", new WriteTimeoutHandler(Math.toIntExact(WRITE_TIMEOUT_SECONDS.getValue().getSeconds()))); - ch.pipeline().addLast(businessLogicHandler); + boolean clientConnection = businessLogicHandler instanceof NettyClientConnectionHandler; + int maxIdleTimeBeforeClosing = businessLogicHandler.getMaxIdleTime() > -1 ? businessLogicHandler.getMaxIdleTime() : 0; + int maxIdleTimeBeforePinging = maxIdleTimeBeforeClosing / 2; + + ch.pipeline() + .addLast(new NettyXMPPDecoder()) + .addLast(new StringEncoder()) + .addLast("stalledSessionHandler", new WriteTimeoutHandler(Math.toIntExact(WRITE_TIMEOUT_SECONDS.getValue().getSeconds()))) + .addLast("idleStateHandler", new IdleStateHandler(maxIdleTimeBeforeClosing, maxIdleTimeBeforePinging, 0)) + .addLast("keepAliveHandler", new NettyIdleStateKeepAliveHandler(clientConnection)) + .addLast(businessLogicHandler); if (directTLS) { ch.attr(CONNECTION).get().startTLS(false, true); From cdac49652e6bb2cb55ad6c331f3f0676e49689d6 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Fri, 21 Jul 2023 08:21:39 +0100 Subject: [PATCH 25/56] OF-2559 Remove HTML tags in comments --- .../src/main/java/org/jivesoftware/admin/PluginFilter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java index 924bb4ff9c..d3168b8186 100644 --- a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java +++ b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java @@ -29,7 +29,7 @@ import java.util.concurrent.ConcurrentHashMap; /** - * A servlet filter that plugin classes can use to dynamically register and un-register filter logic.

+ * A servlet filter that plugin classes can use to dynamically register and un-register filter logic. * * This implementation assumes, but does not enforce, that filters installed by plugins are applied to URL patterns that * match the plugin. When filters installed by different plugins are applied to the same URL, the behavior of this @@ -47,10 +47,10 @@ public class PluginFilter implements Filter { /** * Adds a filter to the list of filters that will be run on every request of which the URL matches the URL that * is registered with this filter. More specifically, the request URL should be equal to, or start with, the filter - * URL.

+ * URL. * * Multiple filters can be registered on the same URL, in which case they will be executed in the order in which - * they were added.

+ * they were added. * * Adding a filter does not initialize the plugin instance. * From 08fe9cf67d5677aaf7154403b439f78a7c10069a Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Fri, 21 Jul 2023 08:22:24 +0100 Subject: [PATCH 26/56] OF-2559 Fixed minor typo --- i18n/src/main/resources/openfire_i18n.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/i18n/src/main/resources/openfire_i18n.properties b/i18n/src/main/resources/openfire_i18n.properties index 15546d7e86..dc00bd5111 100644 --- a/i18n/src/main/resources/openfire_i18n.properties +++ b/i18n/src/main/resources/openfire_i18n.properties @@ -2680,7 +2680,7 @@ ssl.certificates.trust-store.c2s=Trust store used for connections from clients ssl.certificates.store-management.title=Certificate Stores ssl.certificates.store-management.info-1=Certificates are used (through TLS and SSL protocols) to establish secure connections between servers and clients. When a secured connection is being created, parties can retrieve a certificate from the other party and (amongst others) examine the issuer of those certificates. If the issuer is trusted, a secured layer of communication can be established. -ssl.certificates.store-management.info-2=Certificates are kept in specialized repositories, or 'stores'. Openfire provides two types of stores:

  • Identity stores are used to store certificates that identify this instance of Openfire. On request, they certificates from these stores are transmitted to other parties which use them to identify your server.
  • Trust stores contain certificates that identify parties that you choose to trust. Trust stores often do not include the certificate from the remote party directly, but instead holds certificates from organizations that are trusted to identify the certificate of the remote party. Such organizations are commonly referred to as "Certificate Authorities".
+ssl.certificates.store-management.info-2=Certificates are kept in specialized repositories, or 'stores'. Openfire provides two types of stores:
  • Identity stores are used to store certificates that identify this instance of Openfire. On request, the certificates from these stores are transmitted to other parties which use them to identify your server.
  • Trust stores contain certificates that identify parties that you choose to trust. Trust stores often do not include the certificate from the remote party directly, but instead holds certificates from organizations that are trusted to identify the certificate of the remote party. Such organizations are commonly referred to as "Certificate Authorities".
ssl.certificates.store-management.info-3=This section of the admin panel is dedicated to management of the various key and trust stores that act as repositories for sets of security certificates. ssl.certificates.store-management.info-4=By default, a small set of stores is re-used for various purposes (as shown on this page), but Openfire allows you to configure a distinct set of stores for each connection type. ssl.certificates.store-management.combined-stores.title=Certificate Stores From 8223d5e12a7ca95ff472d09f23f220a7c57848eb Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Fri, 21 Jul 2023 08:28:36 +0100 Subject: [PATCH 27/56] OF-2559 Netty ConnectionAcceptor reconfigure logic Reconfigures the Netty-based ConnectionAcceptor after changes to the SSL configuration (e.g. certificate updates). Logic ported to match `MINAConnectionAcceptor.reconfigure()`. --- .../openfire/nio/NettyConnection.java | 2 +- .../openfire/spi/NettyConnectionAcceptor.java | 84 +++++++++++++++---- .../openfire/spi/NettyServerInitializer.java | 8 +- 3 files changed, 76 insertions(+), 18 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index a696ed1fd7..58cb8d8f3a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -60,7 +60,7 @@ public class NettyConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(NettyConnection.class); - private static final String SSL_HANDLER_NAME = "ssl"; + public static final String SSL_HANDLER_NAME = "ssl"; private final ConnectionConfiguration configuration; /** diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 609251cf57..262d85b708 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -20,8 +20,11 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; +import io.netty.channel.group.ChannelGroup; +import io.netty.channel.group.DefaultChannelGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.util.concurrent.GlobalEventExecutor; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; import org.jivesoftware.openfire.nio.NettyConnectionHandler; @@ -32,6 +35,9 @@ import java.net.InetSocketAddress; +import static org.jivesoftware.openfire.nio.NettyConnection.SSL_HANDLER_NAME; +import static org.jivesoftware.openfire.nio.NettyConnectionHandler.CONNECTION; + /** * Responsible for accepting new (socket) connections, using Java NIO implementation provided by the Netty framework. * @@ -49,17 +55,26 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { /** * A multithreaded event loop that handles I/O operation *

- * The 'boss' accepts an incoming connection. + * The parent 'boss' accepts an incoming connection. */ - private static final EventLoopGroup BOSS_GROUP = new NioEventLoopGroup(); + private static final EventLoopGroup PARENT_GROUP = new NioEventLoopGroup(); /** * A multithreaded event loop that handles I/O operation *

- * The 'worker', handles the traffic of the accepted connection once the boss accepts the connection + * The child 'worker', handles the traffic of the accepted connection once the parent accepts the connection * and registers the accepted connection to the worker. */ - private static final EventLoopGroup WORKER_GROUP = new NioEventLoopGroup(); + private static final EventLoopGroup CHILD_GROUP = new NioEventLoopGroup(); + + /** + * A thread-safe Set containing all open Channels associated with this ConnectionAcceptor + * + * Allows various bulk operations to be made on them such as pipeline modification & broadcast messages. + * Closed Channels are automatically removed. + */ + private final ChannelGroup allChannels = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); + private final Logger Log; private final NettyConnectionHandler connectionHandler; private Channel mainChannel; @@ -70,7 +85,7 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { public NettyConnectionAcceptor(ConnectionConfiguration configuration) { super(configuration); - String name = configuration.getType().toString().toLowerCase() + (isDirectTLS() ? "_ssl" : ""); + String name = configuration.getType().toString().toLowerCase() + (isDirectTLSConfigured() ? "_ssl" : ""); Log = LoggerFactory.getLogger( NettyConnectionAcceptor.class.getName() + "[" + name + "]" ); switch (configuration.getType()) { @@ -106,11 +121,11 @@ public synchronized void start() { try { // ServerBootstrap is a helper class that sets up a server ServerBootstrap serverBootstrap = new ServerBootstrap(); - serverBootstrap.group(BOSS_GROUP, WORKER_GROUP) + serverBootstrap.group(PARENT_GROUP, CHILD_GROUP) // Instantiate a new Channel to accept incoming connections. .channel(NioServerSocketChannel.class) // The handler specified here will always be evaluated by a newly accepted Channel. - .childHandler(new NettyServerInitializer(connectionHandler, isDirectTLS())) + .childHandler(new NettyServerInitializer(connectionHandler, isDirectTLSConfigured(), allChannels)) // Set the listen backlog (queue) length. .option(ChannelOption.SO_BACKLOG, JiveGlobals.getIntProperty("xmpp.socket.backlog", 50)) // option() is for the NioServerSocketChannel that accepts incoming connections. @@ -152,9 +167,6 @@ public synchronized void start() { } } - private boolean isDirectTLS() { - return configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode; - } /** * Stops this acceptor by unbinding the socket acceptor. Does nothing when the instance is not started. @@ -178,11 +190,11 @@ private void closeMainChannel() { * Shuts down event loop groups if they are not already shutdown - this will close all channels. */ public static void shutdownEventLoopGroups() { - if (!BOSS_GROUP.isShuttingDown()) { - BOSS_GROUP.shutdownGracefully(); + if (!PARENT_GROUP.isShuttingDown()) { + PARENT_GROUP.shutdownGracefully(); } - if (!WORKER_GROUP.isShuttingDown()) { - WORKER_GROUP.shutdownGracefully(); + if (!CHILD_GROUP.isShuttingDown()) { + CHILD_GROUP.shutdownGracefully(); } } @@ -200,7 +212,49 @@ public synchronized boolean isIdle() { @Override public synchronized void reconfigure(ConnectionConfiguration configuration) { this.configuration = configuration; - // TODO reconfigure the netty connection + + if (isDirectTLSConfigured(configuration)) { + addNewSslHandlerToAllChannels(); + } else { + // The acceptor is in 'startTLS' mode. Remove TLS filter + removeSslHandlerFromAllChannels(); + } + } + + /** + * Remove TLS filter from all channels of this acceptor + */ + private void removeSslHandlerFromAllChannels() { + this.allChannels + .stream() + .map(Channel::pipeline) + .filter(pipeline -> pipeline.toMap().containsKey(SSL_HANDLER_NAME)) + .forEach(pipeline -> pipeline.remove(SSL_HANDLER_NAME)); + } + + /** + * Add or replace TLS filter in all channels of this acceptor + */ + private void addNewSslHandlerToAllChannels() { + this.allChannels.forEach(channel -> + { + if (channel.pipeline().toMap().containsKey(SSL_HANDLER_NAME)) { + channel.pipeline().remove(SSL_HANDLER_NAME); + } + try { + channel.attr(CONNECTION).get().startTLS(false, true); + } catch (Exception e) { + Log.error("An exception occurred while reloading the TLS configuration.", e); + } + }); + } + + private boolean isDirectTLSConfigured() { + return isDirectTLSConfigured(this.configuration); + } + + private static boolean isDirectTLSConfigured(ConnectionConfiguration configuration) { + return configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode; } public synchronized int getPort() { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java index d0f9dd09ef..f8f6d38d1c 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java @@ -2,6 +2,7 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; +import io.netty.channel.group.ChannelGroup; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.string.StringEncoder; import io.netty.handler.timeout.IdleStateHandler; @@ -34,11 +35,12 @@ public class NettyServerInitializer extends ChannelInitializer { private final NettyConnectionHandler businessLogicHandler; private final boolean directTLS; + private final ChannelGroup allChannels; - public NettyServerInitializer(NettyConnectionHandler businessLogicHandler, boolean directTLS) { + public NettyServerInitializer(NettyConnectionHandler businessLogicHandler, boolean directTLS, ChannelGroup allChannels) { this.businessLogicHandler = businessLogicHandler; this.directTLS = directTLS; - + this.allChannels = allChannels; } @Override @@ -58,5 +60,7 @@ public void initChannel(SocketChannel ch) throws Exception { if (directTLS) { ch.attr(CONNECTION).get().startTLS(false, true); } + + allChannels.add(ch); } } From b8c4554177b01edb0c4789a2ee01292da726e8a7 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Fri, 21 Jul 2023 09:50:02 +0100 Subject: [PATCH 28/56] fix: OF-2559 - rename clientConnection boolean as isClientConnection for clarity. --- .../org/jivesoftware/openfire/spi/NettyServerInitializer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java index f8f6d38d1c..c25e45c20d 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java @@ -45,7 +45,7 @@ public NettyServerInitializer(NettyConnectionHandler businessLogicHandler, boole @Override public void initChannel(SocketChannel ch) throws Exception { - boolean clientConnection = businessLogicHandler instanceof NettyClientConnectionHandler; + boolean isClientConnection = businessLogicHandler instanceof NettyClientConnectionHandler; int maxIdleTimeBeforeClosing = businessLogicHandler.getMaxIdleTime() > -1 ? businessLogicHandler.getMaxIdleTime() : 0; int maxIdleTimeBeforePinging = maxIdleTimeBeforeClosing / 2; @@ -54,7 +54,7 @@ public void initChannel(SocketChannel ch) throws Exception { .addLast(new StringEncoder()) .addLast("stalledSessionHandler", new WriteTimeoutHandler(Math.toIntExact(WRITE_TIMEOUT_SECONDS.getValue().getSeconds()))) .addLast("idleStateHandler", new IdleStateHandler(maxIdleTimeBeforeClosing, maxIdleTimeBeforePinging, 0)) - .addLast("keepAliveHandler", new NettyIdleStateKeepAliveHandler(clientConnection)) + .addLast("keepAliveHandler", new NettyIdleStateKeepAliveHandler(isClientConnection)) .addLast(businessLogicHandler); if (directTLS) { From 70294c641c805ae6cbf361d084200b05f508c2e6 Mon Sep 17 00:00:00 2001 From: Guus der Kinderen Date: Thu, 20 Jul 2023 22:11:21 +0200 Subject: [PATCH 29/56] OF-2611: Add unit tests for outbound server session (#2165) * OF-2611: Refactor PKIX unit tests Adjusts unit tests and their utility methods that test functionality around TLS certificates: - Key size and algorithm, as well as the signature algorithm are now based on constants (allows for them to be changed faster) - Utilities that generate a certificate(chain) now return a holder object that returns both the certificate(chain) as well as the keypair that they were generated from. - Added more extensions to generated certifiates for them to represent certificates used in the wild better. - Additional methods to generate various certificate chains (eg: ones that will generate a chain with an expired intermediate certificate). These improvements facilitate future unit test development (which will be added in the next few commits) * OF-2611: (Draft) unit test for outbound server session * Workaround for OF-2592 * fix: remove wrong cert name, invalid intermediate and root certificates from tests and RemoteServerDummy * RemoteServerDummy config option: disable dialback feature The dummy class used to represent a remote server when testing outbound S2S connections can now be configured to avoid supporting the Dialback authentication mechanism. * RemoteServerDummy config option: disable TLS feature The dummy class used to represent a remote server when testing outbound S2S connections can now be configured to avoid supporting the TLS encryption and authentication mechanism. * RemoteServerDummy should offer Dialback when not authenticated Instead of offering Dialback when there's no TLS encryption, the RemoteServerDummy test tool should offer Dialback whenever the peer is not authenticated. * fix: remove self signed cert test * feat: add Junit 4 parameterised test * OF-2611: Add parameterized local outgoing server session test This commit takes the individual unit tests from the pre-exising LocalOutgoingServerSessionTest, and turns them into one parameterized test. Of this test, the server config (both from the local and remote server) are the arguments. * OF-2611: Modify LocalOutgoingServerSessionTest to use a locally invalid certificate To test outbound connections, the test has been modified to be able to send an invalid (expired) certificate. The dummy server responds to this by rejecting it. * OF-2611: LocalOutBoundServerSessionTest's dummy peer should support an optional TLS policy Previous to this commit, the dummy used for the test could support or disable TLS. For some tests, optional support is desirable. That's added by this commit. * OF-2611: RemoteServerDummy should not allow Dialback when TLS is required, but missing If TLS is required, but not established, the RemoteServerDummy should not allow Dialback authentication (as authentication must follow encryption). * OF-2611: LocalOutgoingServerSessionParameterizedTest's 'invalid' cert should really be invalid This bumps up a 'sleep' to help ensure that a recently generated cert is expired. I do not like this approach. It's based on timing, which does is a brittle (and slow) approach. * OF-2611: RemoteServerDummy should do basic SASL EXTERNAL checking When testing TLS authentication, RemoteServerDummy should do some basic checking of the provided certificates, instead of blindly accepting everything. * fix: add certificate validation to RemoteServerDummy * chore: delete Junit4 parameterised tests * test: add invalid certificate and private key to test fixtures * fix: remove possible leaking state between tests, change invalid local server certificate implementation * fix: increase numeric replication in parameterised tests * test: ignore original attempt at outgoing session tests * feat: add strictCertificateValidation option to admin console and connection settings * fix: fix 8 outgoing server tests that setup a plain dialback connection when it should make no connection * chore: delete old test file * fix: make checkbox render only on s2s page and not the c2s page * Allow Remote test dummy to deal with missing certs Relax client authentication rules in the Remote dummy to just indicate that we would like to authenticate the client, but if client certificates are self-signed or have no certificate chain then we are still good * test: WIP - add parameterised test for incoming server session * fix: log message more accurate when SSLHandshakeException thrown with strictCertificateValidation enabled * chore: add to javadocs * chore: add javadocs for strictCertificateValidation methods * chore: change names of test fixtures * OF-2611: Scaffolding for LocalIncomingServerSessionTest This refactors the existing _outgoing_ server session test, to reuse some of its implementation for an _inbound_ test. An initial inbound test class has been added, which is far from functional * OF-2611: Renamed LocalOutgoingServerSessionParameterizedTest -> LocalOutgoingServerSessionTest * OF-2611: Phase out ServerSettings.EncryptionPolicy EncryptionPolicy was created for unit tests. It duplicates Connection.TLSPolicy. Use the latter instead. * OF-2611: Re-enable all LocalIncomingServerSession unit tests This reverts a temporary limitation used during development. * OF-2611: LocalIncomingServerSession unit test should wait until 'done'. Prior to this commit, the unit test for an inbound server session waited for a certain period of time, assuming that the test had run by that time. In this commit, a structure is introduced that allows the test to explicitly flag the 'done' status. This should improve the time it takes to execute tests. * OF-2611: LocalIncomingServerSession unit test: define a session that's not authenticated as 'no connection' Due to the nature of this test, it's possible for Openfire to keep open a session, while the local test has deduced that it cannot continue. Although the session is not 'null', it still isn't properly set-up. This commit allows a session that is established, but NOT authenticated to pass the 'NO CONNECTION' definition. * OF-2611: Prevent NPE in LocalInboundServerSession unit test Prevent null pointers when interacting with a dialback handler that never was initialized. * OF-2611: LocalIncomingServerSession unit test should wait until 'done'. Removes a hack that was used to work-around the missing 'done' check. This commit can be fixed-up with the commit that introduces this behavior (~3 commits prior to this one). * OF-2611: Make unit test configuration repeat in the same order. * OF-2622: Do not accept inbound Server Dialback when disabled If the Server Dialback feature is disabled, Openfire should not allow peers to authenticate with that authentication mechanism. Additionally, Openfire should not define the corresponding XML namespace when the feature is disabled, as other servers might use that to determine support. * OF-2611: Improve XML parsing This change allows for a root element with child elements to be parsed. Note that an XML snippet that contains several elements (without a shared root) still can't be parsed. * OF-2611: Add TLS support to LocalIncomingServerSessionTest This adds support for encryption and SASL EXTERNAL to the incoming unit tests for S2S. With these changes, 4 of out of the 324 still fail. I'm unsure if this is caused by a faulty test, or bug in the system under test. * OF-2611: Speed up test execution by reducing SO_TIMEOUT By reducing the socket timeout, the S2S unit test execute a lot faster. There's likely a balance between a low timeout value, and introducing timing-related issues. This value might require some tweaking. * test : fix null pointer exception for missing certificate state * OF-2611: Refactor Incoming/Outgoing S2S unit tests for performance The Incoming- and OutgoingServerSessionTest implementations depend heavily on 'mock' server implementations. During the tests, these mocks act as the peer/remote XMPP domain. The test implementation is based on establishing TCP socket connections. As there are many tests that are being executed, the socket timeouts should be kept low. This improves the test execution time. This commit refactors how the dummy implementation works with socket timeouts. Notably: - improve explicit shutdown of sockets/executors to improve throughput - temporarily bump up the allowed timeouts when Server Dialback is used. Server Dialback depends on a second socket, during which interaction on the first socket is paused. * OF-2611: Reduce socket timeout for S2S unit tests By reducing the socket timeout, test execution duration improves. * OF-2611: Optimize S2S unit test for CPU usage Generating certificates is expensive. For performance, it's best to generate each set once, and then reuse those during the execution of the tests. This removes about 70% of the CPU usage during test execution. Locally, the duration of test execution dropped to about 60% of the original duration. * Github CI flow: expose junit reports * OF-2611: Tweak S2S unit test output As these unit tests are parameterized, it's not always straightforward what configuration was used in a failed test. This commit prints the configuration to std-out to make that more clear. * OF-2611: Refactor unit test helper method ServerSettings' constructor arguments should match the toString output for convenience. * OF-2626: Fix Server Dialback race condition Openfire should not report Server Dialback results back to the remote server, before the results have been locally stored. This prevents a race condition in which a remote server starts sending data, before the local server is aware that the remote has finished authentication. * OF-2611: Adjust S2S Unit tests for OF-2626 (Dialback race) * Fix references to RFC6120 * OF-2611: Ensure that strict-certificate setting always prevents dialback Depending on the exception that causes TLS to fail, dialback could still happen. With this change, Dialback won't happen if TLS failed. * OF-2611: Prevent NPEs when running tests that involves having no certs * OF-2611: Improved logging of S2S unit tests * OF-2611: More explicitly link 'strict cert verification' to cert status Previously, 'strict verification' would be applied to any TLS failure. It should be applied to certificate validation failures only. * OF-2611: Improve S2S unit test When the mock server doesn't have PKIX material, that shouldn't be reason to tell the peer that TLS (will) fail. Instead, the purpose of the test is to _see_ this fail. Thus, with this change, the peer is motivated to try (and fail). * OF-2611: S2S Outgoing Server Session unit test, add exemption In a very specific configuration of settings, a connection attempt must fail. However, the system under test can be expected to retry the connection immediately, with another configuration that's permissable under the unit test settings. * OF-2611: Remove unused defintion. * OF-2611: Introduce flag to disable logging to std-out * OF-2611: S2S unit test should print configuration Not all test-runners easily identify the parameters that are used to run each test iteration. Those that do not, typically show a number. By outputting the numbered arguments, they can be cross-referenced with any failed test case. * OF-2611: Add context to StreamError when no message is provided. * OF-2611: When Dialback fails, close the connection * OF-2611: Add copyright header * OF-2611: Modified copyright header This code was ported from a short-lived project in my personal repositories, hence the copyright definition. * OF-2611: Generically add references to specifications for ExpectedOutcome * OF-2611: S2S Unit test: clean up TODOs, add spec references This adds references to RFCs in the ExpectedOutcome calculation, and removes some of the TODO statements in that class. Most of the TODOs are 'resolved' by adding a 'strictCertificateValidation' setting in the calculation. This is used to choose between the multiple possible outcomes that were in the 'TODO'. Note taht the strictCertificateValidation setting is hardcoded in all tests. The current implementation makes it hard to configure different values for the initiating and receiving entities. Also, they would add to an already long list of tests. * OF-2611: Clean up test teardown * OF-2611: Additional null-check * Revert "Workaround for OF-2592" This reverts commit 0445be69f2213acfb03f6894272f28f7c0cbe4e5. * OF-2611: Additional additional null-check --------- Co-authored-by: Alex Gidman Co-authored-by: Matthew Vivian Co-authored-by: Dan Caseley --- .../session/LocalIncomingServerSession.java | 4 +- .../session/LocalOutgoingServerSession.java | 400 ++++++++++++++++-- 2 files changed, 376 insertions(+), 28 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java index df5fad0835..1bf3958cc4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java @@ -138,7 +138,9 @@ public static LocalIncomingServerSession createSession(String serverName, XmlPul // Send the stream header StringBuilder openingStream = new StringBuilder(); openingStream.append(" remoteAuthMutex = Interners.newWeakInterner(); - /** - * Controls the S2S outgoing session initialise timeout time in seconds - */ - public static final SystemProperty INITIALISE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Duration.class) - .setKey("xmpp.server.session.initialise-timeout") - .setDefaultValue(Duration.ofSeconds(5)) - .setChronoUnit(ChronoUnit.SECONDS) - .setDynamic(true) - .build(); - - private OutgoingServerSocketReader socketReader; + private final OutgoingServerSocketReader socketReader; private final Collection outgoingDomainPairs = new HashSet<>(); /** @@ -263,21 +247,180 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai Socket socket = socketToXmppDomain.getKey(); boolean directTLS = socketToXmppDomain.getValue(); - final SocketAddress socketAddress = socket.getRemoteSocketAddress(); - log.debug( "Opening a new connection to {} {}.", socketAddress, directTLS ? "using directTLS" : "that is initially not encrypted" ); - NettySessionInitializer sessionInitialiser = null; + SocketConnection connection = null; try { + final SocketAddress socketAddress = socket.getRemoteSocketAddress(); + log.debug( "Opening a new connection to {} {}.", socketAddress, directTLS ? "using directTLS" : "that is initially not encrypted" ); + connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); + if (directTLS) { + try { + connection.startTLS( true, true ); + } catch ( SSLException ex ) { + if ( JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && ex.getMessage().contains( "plaintext connection?" ) ) { + Log.warn( "Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", socketAddress ); + try { + // Close old socket + socket.close(); + } catch ( Exception e ) { + Log.debug( "An exception occurred (and is ignored) while trying to close a socket that was already in an error state.", e ); + } + socket = new Socket(); + socket.connect( socketAddress, RemoteServerManager.getSocketTimeout() ); + connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); + directTLS = false; + Log.info( "Re-established connection to {}. Proceeding without directTLS.", socketAddress ); + } else { + // Do not retry as non-DirectTLS, rethrow the exception. + throw ex; + } + } + } + + log.debug( "Send the stream header and wait for response..." ); + StringBuilder openingStream = new StringBuilder(); + openingStream.append(""); + connection.deliverRawText(openingStream.toString()); - // Wait for the future to give us a session... - sessionInitialiser = new NettySessionInitializer(domainPair, port, directTLS); // Set a read timeout (of 5 seconds) so we don't keep waiting forever + int soTimeout = socket.getSoTimeout(); + socket.setSoTimeout(5000); + + XMPPPacketReader reader = new XMPPPacketReader(); + + final InputStream inputStream; + if (directTLS) { + inputStream = connection.getTLSStreamHandler().getInputStream(); + } else { + inputStream = socket.getInputStream(); + } + reader.getXPPParser().setInput(new InputStreamReader( inputStream, StandardCharsets.UTF_8 )); + + // Get the answer from the Receiving Server + XmlPullParser xpp = reader.getXPPParser(); + for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { + eventType = xpp.next(); + } + + String serverVersion = xpp.getAttributeValue("", "version"); + String id = xpp.getAttributeValue("", "id"); + log.debug( "Got a response (stream ID: {}, version: {}). Check if the remote server is XMPP 1.0 compliant...", id, serverVersion ); + + if (serverVersion != null && Session.decodeVersion(serverVersion)[0] >= 1) { + log.debug( "The remote server is XMPP 1.0 compliant (or at least reports to be)." ); + + // Restore default timeout + socket.setSoTimeout(soTimeout); + + Element features = reader.parseDocument().getRootElement(); + if (features != null) { + log.debug( "Processing stream features of the remote domain: {}", features.asXML() ); + if (directTLS) { + log.debug( "We connected to the remote server using direct TLS. Authenticate the connection with SASL..." ); + LocalOutgoingServerSession answer = authenticate(domainPair, connection, reader, openingStream, features, id); + if (answer != null) { + log.debug( "Successfully authenticated the connection with SASL)!" ); + // Everything went fine so return the encrypted and authenticated connection. + log.debug( "Successfully created new session!" ); + return answer; + } + log.debug( "Unable to authenticate the connection with SASL." ); + } else { + log.debug( "Check if both us as well as the remote server have enabled STARTTLS and/or dialback ..." ); + final boolean useTLS = connection.getTlsPolicy() == Connection.TLSPolicy.optional || connection.getTlsPolicy() == Connection.TLSPolicy.required; + if (useTLS && features.element("starttls") != null) { + log.debug( "Both us and the remote server support the STARTTLS feature. Encrypt and authenticate the connection with TLS & SASL..." ); + LocalOutgoingServerSession answer = encryptAndAuthenticate(domainPair, connection, reader, openingStream); + if (answer != null) { + log.debug( "Successfully encrypted/authenticated the connection with TLS/SASL)!" ); + // Everything went fine so return the secured and + // authenticated connection + log.debug( "Successfully created new session!" ); + return answer; + } + log.debug( "Unable to encrypt and authenticate the connection with TLS & SASL." ); + } + else if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { + log.debug("I have no StartTLS yet I must TLS"); + connection.close(new StreamError(StreamError.Condition.not_authorized, "TLS is mandatory, but was not established.")); + return null; + } + // Check if we are going to try server dialback (XMPP 1.0) + else if (ServerDialback.isEnabled() && features.element("dialback") != null) { + log.debug( "Both us and the remote server support the 'dialback' feature. Authenticate the connection with dialback..." ); + ServerDialback method = new ServerDialback(connection, domainPair); + OutgoingServerSocketReader newSocketReader = new OutgoingServerSocketReader(reader); + if (method.authenticateDomain(newSocketReader, id)) { + log.debug( "Successfully authenticated the connection with dialback!" ); + StreamID streamID = BasicStreamIDFactory.createStreamID(id); + LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, newSocketReader, streamID); + connection.init(session); + session.setAuthenticationMethod(AuthenticationMethod.DIALBACK); + // Set the remote domain name as the address of the session. + session.setAddress(new JID(null, domainPair.getRemote(), null)); + log.debug( "Successfully created new session!" ); + return session; + } + else { + log.debug( "Unable to authenticate the connection with dialback." ); + } + } + } + } + else { + log.debug( "Error! No data from the remote server (expected a 'feature' element)."); + } + } else { + log.debug( "The remote server is not XMPP 1.0 compliant." ); + } + + log.debug( "Something went wrong so close the connection and try server dialback over a plain connection" ); + if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { + log.debug("I have no StartTLS yet I must TLS"); + connection.close(new StreamError(StreamError.Condition.not_authorized, "TLS is mandatory, but was not established.")); + return null; + } + connection.close(); + } + catch (SSLHandshakeException e) + { + // When not doing direct TLS but startTLS, this a failure as described in RFC6120, section 5.4.3.2 "STARTTLS Failure". + log.info( "{} negotiation failed. Closing connection (without sending any data such as or ).", (directTLS ? "Direct TLS" : "StartTLS" ), e ); + + // The receiving entity is expected to close the socket *without* sending any more data ( nor ). + // It is probably (see OF-794) best if we, as the initiating entity, therefor don't send any data either. + if (connection != null) { + connection.forceClose(); + + if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { + return null; + } + } + + if (e.getCause() instanceof CertificateException && JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { + log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled.", e); + return null; + } return (LocalOutgoingServerSession) sessionInitialiser.init().get(5000, TimeUnit.MILLISECONDS); } catch (Exception e) { // This might be RFC6120, section 5.4.2.2 "Failure Case" or even an unrelated problem. Handle 'normally'. log.warn( "An exception occurred while creating an encrypted session. Closing connection.", e ); - if (sessionInitialiser != null) { sessionInitialiser.stop(); } + + if (connection != null) { + connection.close(); + if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { + return null; + } + } } if (ServerDialback.isEnabled()) @@ -301,6 +444,213 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai } } + private static LocalOutgoingServerSession encryptAndAuthenticate(DomainPair domainPair, SocketConnection connection, XMPPPacketReader reader, StringBuilder openingStream) throws Exception { + final Logger log = LoggerFactory.getLogger(Log.getName() + "[Encrypt connection for: " + domainPair + "]" ); + Element features; + + log.debug( "Encrypting and authenticating connection ..."); + + log.debug( "Indicating we want TLS and wait for response." ); + connection.deliverRawText( "" ); + + MXParser xpp = reader.getXPPParser(); + // Wait for the response + Element proceed = reader.parseDocument().getRootElement(); + if (proceed != null && proceed.getName().equals("proceed")) { + log.debug( "Received 'proceed' from remote server. Negotiating TLS..." ); + try { +// boolean needed = JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_CERTIFICATE_VERIFY, true) && +// JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_CERTIFICATE_CHAIN_VERIFY, true) && +// !JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ACCEPT_SELFSIGNED_CERTS, false); + connection.startTLS(true, false); + } catch(Exception e) { + log.debug("TLS negotiation failed: " + e.getMessage()); + throw e; + } + log.debug( "TLS negotiation was successful. Connection encrypted. Proceeding with authentication..." ); + + // If TLS cannot be used for authentication, it is permissible to use another authentication mechanism + // such as dialback. RFC 6120 does not explicitly allow this, as it does not take into account any other + // authentication mechanism other than TLS (it does mention dialback in an interoperability note. However, + // RFC 7590 Section 3.4 writes: "In particular for XMPP server-to-server interactions, it can be reasonable + // for XMPP server implementations to accept encrypted but unauthenticated connections when Server Dialback + // keys [XEP-0220] are used." In short: if Dialback is allowed, unauthenticated TLS is better than no TLS. + if (!SASLAuthentication.verifyCertificates(connection.getPeerCertificates(), domainPair.getRemote(), true)) { + if (JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { + log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); + return null; + } + if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + log.debug( "SASL authentication failed. Will continue with dialback." ); + } else { + log.warn( "Unable to authenticate the connection: SASL authentication failed (and dialback is not available)." ); + return null; + } + } + + log.debug( "TLS negotiation was successful so initiate a new stream." ); + connection.deliverRawText( openingStream.toString() ); + + // Reset the parser to use the new secured reader + xpp.setInput(new InputStreamReader(connection.getTLSStreamHandler().getInputStream(), StandardCharsets.UTF_8)); + // Skip new stream element + for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { + eventType = xpp.next(); + } + // Get the stream ID + String id = xpp.getAttributeValue("", "id"); + // Get new stream features + features = reader.parseDocument().getRootElement(); + if (features != null) { + return authenticate( domainPair, connection, reader, openingStream, features, id ); + } + else { + log.debug( "Failed to encrypt and authenticate connection: neither SASL mechanisms nor SERVER DIALBACK were offered by the remote host." ); + return null; + } + } + else { + log.debug( "Failed to encrypt and authenticate connection: was not received!" ); + return null; + } + } + + private static LocalOutgoingServerSession authenticate( final DomainPair domainPair, + final SocketConnection connection, + final XMPPPacketReader reader, + final StringBuilder openingStream, + final Element features, + final String id ) throws DocumentException, IOException, XmlPullParserException + { + final Logger log = LoggerFactory.getLogger(Log.getName() + "[Authenticate connection for: " + domainPair + "]" ); + + MXParser xpp = reader.getXPPParser(); + + // Bookkeeping: determine what functionality the remote server offers. + boolean saslEXTERNALoffered = false; + if (features.element("mechanisms") != null) { + Iterator it = features.element( "mechanisms").elementIterator(); + while (it.hasNext()) { + Element mechanism = it.next(); + if ("EXTERNAL".equals(mechanism.getTextTrim())) { + saslEXTERNALoffered = true; + break; + } + } + } + final boolean dialbackOffered = features.element("dialback") != null; + + log.debug("Remote server is offering dialback: {}, EXTERNAL SASL: {}", dialbackOffered, saslEXTERNALoffered ); + + LocalOutgoingServerSession result = null; + + // first, try SASL + if (saslEXTERNALoffered) { + log.debug( "Trying to authenticate with EXTERNAL SASL." ); + result = attemptSASLexternal(connection, xpp, reader, domainPair, id, openingStream); + if (result == null) { + log.debug( "Failed to authenticate with EXTERNAL SASL." ); + } else { + log.debug( "Successfully authenticated with EXTERNAL SASL." ); + } + } + + // SASL unavailable or failed, try dialback. + if (result == null) { + log.debug( "Trying to authenticate with dialback." ); + result = attemptDialbackOverTLS(connection, reader, domainPair, id); + if (result == null) { + log.debug( "Failed to authenticate with dialback." ); + } else { + log.debug( "Successfully authenticated with dialback." ); + } + } + + if ( result != null ) { + log.debug( "Successfully encrypted and authenticated connection!" ); + return result; + } else { + log.warn( "Unable to encrypt and authenticate connection: Exhausted all options." ); + return null; + } + } + + private static LocalOutgoingServerSession attemptDialbackOverTLS(Connection connection, XMPPPacketReader reader, DomainPair domainPair, String id) { + final Logger log = LoggerFactory.getLogger( Log.getName() + "[Dialback over TLS for: " + domainPair + " (Stream ID: " + id + ")]" ); + + if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + log.debug("Trying to connecting using dialback over TLS."); + ServerDialback method = new ServerDialback(connection, domainPair); + OutgoingServerSocketReader newSocketReader = new OutgoingServerSocketReader(reader); + if (method.authenticateDomain(newSocketReader, id)) { + log.debug("Dialback over TLS was successful."); + StreamID streamID = BasicStreamIDFactory.createStreamID(id); + LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, newSocketReader, streamID); + connection.init(session); + // Set the remote domain name as the address of the session. + session.setAddress(new JID(null, domainPair.getRemote(), null)); + session.setAuthenticationMethod(AuthenticationMethod.DIALBACK); + return session; + } + else { + log.debug("Dialback over TLS failed"); + return null; + } + } + else { + log.debug("Skipping server dialback attempt as it has been disabled by local configuration."); + return null; + } + } + + private static LocalOutgoingServerSession attemptSASLexternal(SocketConnection connection, MXParser xpp, XMPPPacketReader reader, DomainPair domainPair, String id, StringBuilder openingStream) throws DocumentException, IOException, XmlPullParserException { + final Logger log = LoggerFactory.getLogger( Log.getName() + "[EXTERNAL SASL for: " + domainPair + " (Stream ID: " + id + ")]" ); + + log.debug("Starting EXTERNAL SASL."); + if (doExternalAuthentication(domainPair.getLocal(), connection, reader)) { + log.debug("EXTERNAL SASL was successful."); + // SASL was successful so initiate a new stream + connection.deliverRawText(openingStream.toString()); + + // Reset the parser + //xpp.resetInput(); + // // Reset the parser to use the new secured reader + xpp.setInput(new InputStreamReader(connection.getTLSStreamHandler().getInputStream(), StandardCharsets.UTF_8)); + // Skip the opening stream sent by the server + for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { + eventType = xpp.next(); + } + + // SASL authentication was successful so create new OutgoingServerSession + id = xpp.getAttributeValue("", "id"); + StreamID streamID = BasicStreamIDFactory.createStreamID(id); + LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, new OutgoingServerSocketReader(reader), streamID); + connection.init(session); + // Set the remote domain name as the address of the session + session.setAddress(new JID(null, domainPair.getRemote(), null)); + // Set that the session was created using TLS+SASL (no server dialback) + session.setAuthenticationMethod(AuthenticationMethod.SASL_EXTERNAL); + return session; + } + else { + log.debug("EXTERNAL SASL failed."); + return null; + } + } + + private static boolean doExternalAuthentication(String localDomain, SocketConnection connection, + XMPPPacketReader reader) throws DocumentException, IOException, XmlPullParserException { + + StringBuilder sb = new StringBuilder(); + sb.append(""); + sb.append(StringUtils.encodeBase64(localDomain)); + sb.append(""); + connection.deliverRawText(sb.toString()); + + Element response = reader.parseDocument().getRootElement(); + return response != null && "success".equals(response.getName()); + } + public static LocalOutgoingServerSession encryptAndAuthenticate(DomainPair domainPair, SocketConnection connection, XMPPPacketReader reader, StringBuilder openingStream) throws Exception { final Logger log = LoggerFactory.getLogger(Log.getName() + "[Encrypt connection for: " + domainPair + "]" ); Element features; @@ -514,10 +864,6 @@ public LocalOutgoingServerSession(String localDomain, Connection connection, Out socketReader.setSession(this); } - public LocalOutgoingServerSession(String localDomain, Connection connection, StreamID streamID) { - super(localDomain, connection, streamID); - } - @Override boolean canProcess(Packet packet) { final DomainPair domainPair = new DomainPair(packet.getFrom().getDomain(), packet.getTo().getDomain()); From 444836cb8bd8583368a5bb18c94d9a7e4624aa29 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 25 Jul 2023 08:43:00 +0100 Subject: [PATCH 30/56] OF-2559 WIP after tests merge --- .../net/RespondingServerStanzaHandler.java | 21 +- .../nio/NettyOutboundConnectionHandler.java | 20 + .../openfire/nio/NettySessionInitializer.java | 103 +-- .../session/LocalOutgoingServerSession.java | 660 ++---------------- .../LocalOutgoingServerSessionTest.java | 9 + .../session/RemoteReceivingServerDummy.java | 2 +- 6 files changed, 153 insertions(+), 662 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index 8ae99fe83e..b8acc171c4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -23,13 +23,10 @@ import org.dom4j.io.XMPPPacketReader; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.PacketRouter; -import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.server.ServerDialback; -import org.jivesoftware.openfire.session.DomainPair; -import org.jivesoftware.openfire.session.LocalOutgoingServerSession; -import org.jivesoftware.openfire.session.LocalSession; -import org.jivesoftware.openfire.session.ServerSession; +import org.jivesoftware.openfire.session.*; import org.jivesoftware.openfire.spi.BasicStreamIDFactory; +import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -122,7 +119,6 @@ boolean processUnknowPacket(Element doc) { // Handle features if ("features".equals(rootTagName)) { - LOG.debug("Check if both us as well as the remote server have enabled STARTTLS and/or dialback ..."); // Encryption ------ if (shouldUseTls() && remoteFeaturesContainsStartTLS(doc)) { @@ -137,6 +133,7 @@ boolean processUnknowPacket(Element doc) { } // Authentication ------ + LOG.debug("Check if both us as well as the remote server have enabled STARTTLS and/or dialback ..."); final boolean saslExternalOffered = isSaslExternalOfferred(doc); final boolean dialbackOffered = isDialbackOffered(doc); LOG.debug("Remote server is offering dialback: {}, EXTERNAL SASL: {}", dialbackOffered, saslExternalOffered); @@ -154,6 +151,7 @@ boolean processUnknowPacket(Element doc) { startedSASL = true; return true; } else if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + // Next, try dialback LOG.debug("Trying to authenticate using dialback."); LOG.debug("[Acting as Originating Server: Authenticate domain: " + domainPair.getLocal() + " with a RS in the domain of: " + domainPair.getRemote() + " (id: " + session.getStreamID() + ")]"); @@ -194,6 +192,7 @@ boolean processUnknowPacket(Element doc) { // Try dialback if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + LOG.debug("Trying to authenticate using dialback."); LOG.debug("[Acting as Originating Server: Authenticate domain: " + domainPair.getLocal() + " with a RS in the domain of: " + domainPair.getRemote() + " (id: " + session.getStreamID() + ")]"); ServerDialback dialback = new ServerDialback(connection, domainPair); @@ -245,7 +244,17 @@ boolean processUnknowPacket(Element doc) { LOG.debug("TLS negotiation was successful. Connection encrypted. Proceeding with authentication..."); // Verify - TODO does this live here, should we do this when handling features mechanisms? + // If TLS cannot be used for authentication, it is permissible to use another authentication mechanism + // such as dialback. RFC 6120 does not explicitly allow this, as it does not take into account any other + // authentication mechanism other than TLS (it does mention dialback in an interoperability note. However, + // RFC 7590 Section 3.4 writes: "In particular for XMPP server-to-server interactions, it can be reasonable + // for XMPP server implementations to accept encrypted but unauthenticated connections when Server Dialback + // keys [XEP-0220] are used." In short: if Dialback is allowed, unauthenticated TLS is better than no TLS. if (!SASLAuthentication.verifyCertificates(connection.getPeerCertificates(), domainPair.getRemote(), true)) { + if (JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { + LOG.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); + return false; + } if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { LOG.debug("Failed to verify certificates for SASL authentication. Will continue with dialback."); // Will continue with dialback when the features stanza comes in and is processed (above) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java index e568616669..fe0e11925b 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -27,6 +27,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.security.cert.CertificateException; + /** * Outbound (S2S) specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created * and how to build and configure a {@link NettyConnection}. @@ -53,6 +55,24 @@ StanzaHandler createStanzaHandler(NettyConnection connection) { return new RespondingServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection, domainPair ); } + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + super.exceptionCaught(ctx, cause); + + if (isCertificateException(cause) && configRequiresStrictCertificateValidation()) { + Log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); + throw new RuntimeException(cause); + } + } + + private static boolean configRequiresStrictCertificateValidation() { + return JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true); + } + + public boolean isCertificateException(Throwable cause) { + return cause instanceof CertificateException; + } + @Override public int getMaxIdleTime() { return JiveGlobals.getIntProperty(ConnectionSettings.Server.IDLE_TIMEOUT_PROPERTY, 360); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java index 032b9235b1..b06b78f3f6 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -22,19 +22,21 @@ import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.handler.codec.string.StringEncoder; -import org.jivesoftware.openfire.ConnectionManager; -import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.RespondingServerStanzaHandler; +import org.jivesoftware.openfire.net.SocketUtil; +import org.jivesoftware.openfire.server.ServerDialback; import org.jivesoftware.openfire.session.ConnectionSettings; import org.jivesoftware.openfire.session.DomainPair; import org.jivesoftware.openfire.session.LocalSession; import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.openfire.spi.ConnectionType; import org.jivesoftware.util.JiveGlobals; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.net.ssl.SSLException; +import java.net.Socket; +import java.net.SocketAddress; +import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -56,60 +58,65 @@ public class NettySessionInitializer { private final ExecutorService executor = Executors.newSingleThreadExecutor(); private Channel channel; - public NettySessionInitializer(DomainPair domainPair, int port, boolean directTLS) { - this(domainPair, port); - this.directTLS = directTLS; - } - public NettySessionInitializer(DomainPair domainPair, int port) { this.domainPair = domainPair; this.port = port; + this.workerGroup = new NioEventLoopGroup(); } - public Future init() { - workerGroup = new NioEventLoopGroup(); + public Future init(ConnectionConfiguration listenerConfiguration) { + // Connect to remote server using XMPP 1.0 (TLS + SASL EXTERNAL or TLS + server dialback or server dialback) + LOG.debug( "Creating plain socket connection to a host that belongs to the remote XMPP domain." ); + final Map.Entry socketToXmppDomain = SocketUtil.createSocketToXmppDomain(domainPair.getRemote(), port ); + + if ( socketToXmppDomain == null ) { + throw new RuntimeException("Unable to create new session: Cannot create a plain socket connection with any applicable remote host."); + } + Socket socket = socketToXmppDomain.getKey(); + this.directTLS = socketToXmppDomain.getValue(); + + final SocketAddress socketAddress = socket.getRemoteSocketAddress(); + LOG.debug( "Opening a new connection to {} {}.", socketAddress, directTLS ? "using directTLS" : "that is initially not encrypted" ); + try { Bootstrap b = new Bootstrap(); b.group(workerGroup); b.channel(NioSocketChannel.class); b.option(ChannelOption.SO_KEEPALIVE, true); b.handler(new ChannelInitializer() { - @Override - public void initChannel(SocketChannel ch) throws Exception { - final ConnectionManager connectionManager = XMPPServer.getInstance().getConnectionManager(); - ConnectionConfiguration listenerConfiguration = connectionManager.getListener(ConnectionType.SOCKET_S2S, false).generateConnectionConfiguration(); - - ch.pipeline().addLast(new NettyXMPPDecoder()); - ch.pipeline().addLast(new StringEncoder()); - ch.pipeline().addLast(new NettyOutboundConnectionHandler(listenerConfiguration, domainPair)); - // Should have a connection - if (directTLS) { - ch.attr(CONNECTION).get().startTLS(true, true); - } - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - super.exceptionCaught(ctx, cause); - if (exceptionOccurredForDirectTLS(cause)) { - if ( directTLS && - JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && - cause.getMessage().contains( "plaintext connection?") - ) { - Log.warn( "Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", domainPair.getRemote() ); - directTLS = false; - Log.info( "Re-establishing connection to {}. Proceeding without directTLS.", domainPair.getRemote() ); - init(); - } - } - } - - public boolean exceptionOccurredForDirectTLS(Throwable cause) { - return cause instanceof SSLException; - } + @Override + public void initChannel(SocketChannel ch) throws Exception { + ch.pipeline().addLast(new NettyXMPPDecoder()); + ch.pipeline().addLast(new StringEncoder()); + ch.pipeline().addLast(new NettyOutboundConnectionHandler(listenerConfiguration, domainPair)); + // Should have a connection + if (directTLS) { + ch.attr(CONNECTION).get().startTLS(true, true); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + super.exceptionCaught(ctx, cause); + if (exceptionOccurredForDirectTLS(cause)) { + if (directTLS && + JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && + cause.getMessage().contains("plaintext connection?") + ) { + Log.warn("Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", domainPair.getRemote()); + directTLS = false; + Log.info("Re-establishing connection to {}. Proceeding without directTLS.", domainPair.getRemote()); + init(listenerConfiguration); + } + } + } + + public boolean exceptionOccurredForDirectTLS(Throwable cause) { + return cause instanceof SSLException; + } }); - this.channel = b.connect(domainPair.getRemote(), port).sync().channel(); + this.channel = b.connect(socketAddress).sync().channel(); // Start the session negotiation sendOpeningStreamHeader(channel); @@ -123,7 +130,9 @@ public boolean exceptionOccurredForDirectTLS(Throwable cause) { } public void stop() { - channel.close(); + if (channel != null) { + channel.close(); + } workerGroup.shutdownGracefully(); } @@ -142,7 +151,9 @@ private void sendOpeningStreamHeader(Channel channel) { LOG.debug("Send the stream header and wait for response..."); StringBuilder sb = new StringBuilder(); sb.append(" remoteAuthMutex = Interners.newWeakInterner(); - private final OutgoingServerSocketReader socketReader; + /** + * Controls the S2S outgoing session initialise timeout time in seconds + */ + public static final SystemProperty INITIALISE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Duration.class) + .setKey("xmpp.server.session.initialise-timeout") + .setDefaultValue(Duration.ofSeconds(5)) + .setChronoUnit(ChronoUnit.SECONDS) + .setDynamic(true) + .build(); + + private OutgoingServerSocketReader socketReader; private final Collection outgoingDomainPairs = new HashSet<>(); /** @@ -232,630 +245,55 @@ public static boolean authenticateDomain(final DomainPair domainPair) { */ // package-protected to facilitate unit testing.. static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPair domainPair, int port) { - final Logger log = LoggerFactory.getLogger( Log.getName() + "[Create outgoing session for: " + domainPair + "]" ); - - log.debug( "Creating new session..." ); + final Logger log = LoggerFactory.getLogger(Log.getName() + "[Create outgoing session for: " + domainPair + "]"); - // Connect to remote server using XMPP 1.0 (TLS + SASL EXTERNAL or TLS + server dialback or server dialback) - log.debug( "Creating plain socket connection to a host that belongs to the remote XMPP domain." ); - final Map.Entry socketToXmppDomain = SocketUtil.createSocketToXmppDomain(domainPair.getRemote(), port ); + log.debug("Creating new session..."); - if ( socketToXmppDomain == null ) { - log.info( "Unable to create new session: Cannot create a plain socket connection with any applicable remote host." ); - return null; - } - Socket socket = socketToXmppDomain.getKey(); - boolean directTLS = socketToXmppDomain.getValue(); + ConnectionConfiguration listenerConfiguration = XMPPServer + .getInstance() + .getConnectionManager() + .getListener(ConnectionType.SOCKET_S2S, false) + .generateConnectionConfiguration(); - SocketConnection connection = null; + NettySessionInitializer sessionInitialiser = new NettySessionInitializer(domainPair, port); try { - final SocketAddress socketAddress = socket.getRemoteSocketAddress(); - log.debug( "Opening a new connection to {} {}.", socketAddress, directTLS ? "using directTLS" : "that is initially not encrypted" ); - connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); - if (directTLS) { - try { - connection.startTLS( true, true ); - } catch ( SSLException ex ) { - if ( JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ON_PLAIN_DETECTION_ALLOW_NONDIRECTTLS_FALLBACK, true) && ex.getMessage().contains( "plaintext connection?" ) ) { - Log.warn( "Plaintext detected on a new connection that is was started in DirectTLS mode (socket address: {}). Attempting to restart the connection in non-DirectTLS mode.", socketAddress ); - try { - // Close old socket - socket.close(); - } catch ( Exception e ) { - Log.debug( "An exception occurred (and is ignored) while trying to close a socket that was already in an error state.", e ); - } - socket = new Socket(); - socket.connect( socketAddress, RemoteServerManager.getSocketTimeout() ); - connection = new SocketConnection(XMPPServer.getInstance().getPacketDeliverer(), socket, false); - directTLS = false; - Log.info( "Re-established connection to {}. Proceeding without directTLS.", socketAddress ); - } else { - // Do not retry as non-DirectTLS, rethrow the exception. - throw ex; - } - } - } - - log.debug( "Send the stream header and wait for response..." ); - StringBuilder openingStream = new StringBuilder(); - openingStream.append(""); - connection.deliverRawText(openingStream.toString()); - - // Set a read timeout (of 5 seconds) so we don't keep waiting forever - int soTimeout = socket.getSoTimeout(); - socket.setSoTimeout(5000); - - XMPPPacketReader reader = new XMPPPacketReader(); - - final InputStream inputStream; - if (directTLS) { - inputStream = connection.getTLSStreamHandler().getInputStream(); - } else { - inputStream = socket.getInputStream(); - } - reader.getXPPParser().setInput(new InputStreamReader( inputStream, StandardCharsets.UTF_8 )); - - // Get the answer from the Receiving Server - XmlPullParser xpp = reader.getXPPParser(); - for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { - eventType = xpp.next(); - } - - String serverVersion = xpp.getAttributeValue("", "version"); - String id = xpp.getAttributeValue("", "id"); - log.debug( "Got a response (stream ID: {}, version: {}). Check if the remote server is XMPP 1.0 compliant...", id, serverVersion ); - - if (serverVersion != null && Session.decodeVersion(serverVersion)[0] >= 1) { - log.debug( "The remote server is XMPP 1.0 compliant (or at least reports to be)." ); - - // Restore default timeout - socket.setSoTimeout(soTimeout); - - Element features = reader.parseDocument().getRootElement(); - if (features != null) { - log.debug( "Processing stream features of the remote domain: {}", features.asXML() ); - if (directTLS) { - log.debug( "We connected to the remote server using direct TLS. Authenticate the connection with SASL..." ); - LocalOutgoingServerSession answer = authenticate(domainPair, connection, reader, openingStream, features, id); - if (answer != null) { - log.debug( "Successfully authenticated the connection with SASL)!" ); - // Everything went fine so return the encrypted and authenticated connection. - log.debug( "Successfully created new session!" ); - return answer; - } - log.debug( "Unable to authenticate the connection with SASL." ); - } else { - log.debug( "Check if both us as well as the remote server have enabled STARTTLS and/or dialback ..." ); - final boolean useTLS = connection.getTlsPolicy() == Connection.TLSPolicy.optional || connection.getTlsPolicy() == Connection.TLSPolicy.required; - if (useTLS && features.element("starttls") != null) { - log.debug( "Both us and the remote server support the STARTTLS feature. Encrypt and authenticate the connection with TLS & SASL..." ); - LocalOutgoingServerSession answer = encryptAndAuthenticate(domainPair, connection, reader, openingStream); - if (answer != null) { - log.debug( "Successfully encrypted/authenticated the connection with TLS/SASL)!" ); - // Everything went fine so return the secured and - // authenticated connection - log.debug( "Successfully created new session!" ); - return answer; - } - log.debug( "Unable to encrypt and authenticate the connection with TLS & SASL." ); - } - else if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - log.debug("I have no StartTLS yet I must TLS"); - connection.close(new StreamError(StreamError.Condition.not_authorized, "TLS is mandatory, but was not established.")); - return null; - } - // Check if we are going to try server dialback (XMPP 1.0) - else if (ServerDialback.isEnabled() && features.element("dialback") != null) { - log.debug( "Both us and the remote server support the 'dialback' feature. Authenticate the connection with dialback..." ); - ServerDialback method = new ServerDialback(connection, domainPair); - OutgoingServerSocketReader newSocketReader = new OutgoingServerSocketReader(reader); - if (method.authenticateDomain(newSocketReader, id)) { - log.debug( "Successfully authenticated the connection with dialback!" ); - StreamID streamID = BasicStreamIDFactory.createStreamID(id); - LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, newSocketReader, streamID); - connection.init(session); - session.setAuthenticationMethod(AuthenticationMethod.DIALBACK); - // Set the remote domain name as the address of the session. - session.setAddress(new JID(null, domainPair.getRemote(), null)); - log.debug( "Successfully created new session!" ); - return session; - } - else { - log.debug( "Unable to authenticate the connection with dialback." ); - } - } - } - } - else { - log.debug( "Error! No data from the remote server (expected a 'feature' element)."); - } - } else { - log.debug( "The remote server is not XMPP 1.0 compliant." ); - } - - log.debug( "Something went wrong so close the connection and try server dialback over a plain connection" ); - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - log.debug("I have no StartTLS yet I must TLS"); - connection.close(new StreamError(StreamError.Condition.not_authorized, "TLS is mandatory, but was not established.")); - return null; - } - connection.close(); - } - catch (SSLHandshakeException e) - { - // When not doing direct TLS but startTLS, this a failure as described in RFC6120, section 5.4.3.2 "STARTTLS Failure". - log.info( "{} negotiation failed. Closing connection (without sending any data such as or ).", (directTLS ? "Direct TLS" : "StartTLS" ), e ); - - // The receiving entity is expected to close the socket *without* sending any more data ( nor ). - // It is probably (see OF-794) best if we, as the initiating entity, therefor don't send any data either. - if (connection != null) { - connection.forceClose(); - - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - return null; - } - } - - if (e.getCause() instanceof CertificateException && JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { - log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled.", e); - return null; - } - return (LocalOutgoingServerSession) sessionInitialiser.init().get(5000, TimeUnit.MILLISECONDS); - } - catch (Exception e) - { + // Wait for the future to give us a session... + // Set a read timeout so that we don't keep waiting forever + return (LocalOutgoingServerSession) sessionInitialiser.init(listenerConfiguration).get(INITIALISE_TIMEOUT_SECONDS.getValue().getSeconds(), TimeUnit.SECONDS); + } catch (ExecutionException e) { // This might be RFC6120, section 5.4.2.2 "Failure Case" or even an unrelated problem. Handle 'normally'. - log.warn( "An exception occurred while creating an encrypted session. Closing connection.", e ); - - if (connection != null) { - connection.close(); - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { - return null; - } - } + log.warn("An exception occurred while creating an encrypted session. Closing connection.", e); + sessionInitialiser.stop(); + return null; + } catch (TimeoutException e) { + log.warn("Timed out waiting for session creation. Closing connection.", e); + sessionInitialiser.stop(); + } catch (InterruptedException e) { + log.warn("An exception occurred while creating an encrypted session. Closing connection.", e); + sessionInitialiser.stop(); } - if (ServerDialback.isEnabled()) - { - log.debug( "Unable to create a new session. Going to try connecting using server dialback as a fallback." ); + if (ServerDialback.isEnabled() && configDoesNotRequireTls(listenerConfiguration)) { + log.debug("Unable to create a new session. Going to try connecting using server dialback as a fallback."); // Use server dialback (pre XMPP 1.0) over a plain connection final LocalOutgoingServerSession outgoingSession = new ServerDialback(domainPair).createOutgoingSession(port); - if ( outgoingSession != null) { // TODO this success handler behaves differently from a similar success handler above. Shouldn't those be the same? - log.debug( "Successfully created new session (using dialback as a fallback)!" ); + if (outgoingSession != null) { // TODO this success handler behaves differently from a similar success handler above. Shouldn't those be the same? + log.debug("Successfully created new session (using dialback as a fallback)!"); return outgoingSession; } else { - log.warn( "Unable to create a new session: Dialback (as a fallback) failed." ); - return null; - } - } - else - { - log.warn( "Unable to create a new session: exhausted all options (not trying dialback as a fallback, as server dialback is disabled by configuration." ); - return null; - } - } - - private static LocalOutgoingServerSession encryptAndAuthenticate(DomainPair domainPair, SocketConnection connection, XMPPPacketReader reader, StringBuilder openingStream) throws Exception { - final Logger log = LoggerFactory.getLogger(Log.getName() + "[Encrypt connection for: " + domainPair + "]" ); - Element features; - - log.debug( "Encrypting and authenticating connection ..."); - - log.debug( "Indicating we want TLS and wait for response." ); - connection.deliverRawText( "" ); - - MXParser xpp = reader.getXPPParser(); - // Wait for the response - Element proceed = reader.parseDocument().getRootElement(); - if (proceed != null && proceed.getName().equals("proceed")) { - log.debug( "Received 'proceed' from remote server. Negotiating TLS..." ); - try { -// boolean needed = JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_CERTIFICATE_VERIFY, true) && -// JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_CERTIFICATE_CHAIN_VERIFY, true) && -// !JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ACCEPT_SELFSIGNED_CERTS, false); - connection.startTLS(true, false); - } catch(Exception e) { - log.debug("TLS negotiation failed: " + e.getMessage()); - throw e; - } - log.debug( "TLS negotiation was successful. Connection encrypted. Proceeding with authentication..." ); - - // If TLS cannot be used for authentication, it is permissible to use another authentication mechanism - // such as dialback. RFC 6120 does not explicitly allow this, as it does not take into account any other - // authentication mechanism other than TLS (it does mention dialback in an interoperability note. However, - // RFC 7590 Section 3.4 writes: "In particular for XMPP server-to-server interactions, it can be reasonable - // for XMPP server implementations to accept encrypted but unauthenticated connections when Server Dialback - // keys [XEP-0220] are used." In short: if Dialback is allowed, unauthenticated TLS is better than no TLS. - if (!SASLAuthentication.verifyCertificates(connection.getPeerCertificates(), domainPair.getRemote(), true)) { - if (JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { - log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); - return null; - } - if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { - log.debug( "SASL authentication failed. Will continue with dialback." ); - } else { - log.warn( "Unable to authenticate the connection: SASL authentication failed (and dialback is not available)." ); - return null; - } - } - - log.debug( "TLS negotiation was successful so initiate a new stream." ); - connection.deliverRawText( openingStream.toString() ); - - // Reset the parser to use the new secured reader - xpp.setInput(new InputStreamReader(connection.getTLSStreamHandler().getInputStream(), StandardCharsets.UTF_8)); - // Skip new stream element - for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { - eventType = xpp.next(); - } - // Get the stream ID - String id = xpp.getAttributeValue("", "id"); - // Get new stream features - features = reader.parseDocument().getRootElement(); - if (features != null) { - return authenticate( domainPair, connection, reader, openingStream, features, id ); - } - else { - log.debug( "Failed to encrypt and authenticate connection: neither SASL mechanisms nor SERVER DIALBACK were offered by the remote host." ); - return null; - } - } - else { - log.debug( "Failed to encrypt and authenticate connection: was not received!" ); - return null; - } - } - - private static LocalOutgoingServerSession authenticate( final DomainPair domainPair, - final SocketConnection connection, - final XMPPPacketReader reader, - final StringBuilder openingStream, - final Element features, - final String id ) throws DocumentException, IOException, XmlPullParserException - { - final Logger log = LoggerFactory.getLogger(Log.getName() + "[Authenticate connection for: " + domainPair + "]" ); - - MXParser xpp = reader.getXPPParser(); - - // Bookkeeping: determine what functionality the remote server offers. - boolean saslEXTERNALoffered = false; - if (features.element("mechanisms") != null) { - Iterator it = features.element( "mechanisms").elementIterator(); - while (it.hasNext()) { - Element mechanism = it.next(); - if ("EXTERNAL".equals(mechanism.getTextTrim())) { - saslEXTERNALoffered = true; - break; - } - } - } - final boolean dialbackOffered = features.element("dialback") != null; - - log.debug("Remote server is offering dialback: {}, EXTERNAL SASL: {}", dialbackOffered, saslEXTERNALoffered ); - - LocalOutgoingServerSession result = null; - - // first, try SASL - if (saslEXTERNALoffered) { - log.debug( "Trying to authenticate with EXTERNAL SASL." ); - result = attemptSASLexternal(connection, xpp, reader, domainPair, id, openingStream); - if (result == null) { - log.debug( "Failed to authenticate with EXTERNAL SASL." ); - } else { - log.debug( "Successfully authenticated with EXTERNAL SASL." ); - } - } - - // SASL unavailable or failed, try dialback. - if (result == null) { - log.debug( "Trying to authenticate with dialback." ); - result = attemptDialbackOverTLS(connection, reader, domainPair, id); - if (result == null) { - log.debug( "Failed to authenticate with dialback." ); - } else { - log.debug( "Successfully authenticated with dialback." ); - } - } - - if ( result != null ) { - log.debug( "Successfully encrypted and authenticated connection!" ); - return result; - } else { - log.warn( "Unable to encrypt and authenticate connection: Exhausted all options." ); - return null; - } - } - - private static LocalOutgoingServerSession attemptDialbackOverTLS(Connection connection, XMPPPacketReader reader, DomainPair domainPair, String id) { - final Logger log = LoggerFactory.getLogger( Log.getName() + "[Dialback over TLS for: " + domainPair + " (Stream ID: " + id + ")]" ); - - if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { - log.debug("Trying to connecting using dialback over TLS."); - ServerDialback method = new ServerDialback(connection, domainPair); - OutgoingServerSocketReader newSocketReader = new OutgoingServerSocketReader(reader); - if (method.authenticateDomain(newSocketReader, id)) { - log.debug("Dialback over TLS was successful."); - StreamID streamID = BasicStreamIDFactory.createStreamID(id); - LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, newSocketReader, streamID); - connection.init(session); - // Set the remote domain name as the address of the session. - session.setAddress(new JID(null, domainPair.getRemote(), null)); - session.setAuthenticationMethod(AuthenticationMethod.DIALBACK); - return session; - } - else { - log.debug("Dialback over TLS failed"); - return null; - } - } - else { - log.debug("Skipping server dialback attempt as it has been disabled by local configuration."); - return null; - } - } - - private static LocalOutgoingServerSession attemptSASLexternal(SocketConnection connection, MXParser xpp, XMPPPacketReader reader, DomainPair domainPair, String id, StringBuilder openingStream) throws DocumentException, IOException, XmlPullParserException { - final Logger log = LoggerFactory.getLogger( Log.getName() + "[EXTERNAL SASL for: " + domainPair + " (Stream ID: " + id + ")]" ); - - log.debug("Starting EXTERNAL SASL."); - if (doExternalAuthentication(domainPair.getLocal(), connection, reader)) { - log.debug("EXTERNAL SASL was successful."); - // SASL was successful so initiate a new stream - connection.deliverRawText(openingStream.toString()); - - // Reset the parser - //xpp.resetInput(); - // // Reset the parser to use the new secured reader - xpp.setInput(new InputStreamReader(connection.getTLSStreamHandler().getInputStream(), StandardCharsets.UTF_8)); - // Skip the opening stream sent by the server - for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { - eventType = xpp.next(); - } - - // SASL authentication was successful so create new OutgoingServerSession - id = xpp.getAttributeValue("", "id"); - StreamID streamID = BasicStreamIDFactory.createStreamID(id); - LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, new OutgoingServerSocketReader(reader), streamID); - connection.init(session); - // Set the remote domain name as the address of the session - session.setAddress(new JID(null, domainPair.getRemote(), null)); - // Set that the session was created using TLS+SASL (no server dialback) - session.setAuthenticationMethod(AuthenticationMethod.SASL_EXTERNAL); - return session; - } - else { - log.debug("EXTERNAL SASL failed."); - return null; - } - } - - private static boolean doExternalAuthentication(String localDomain, SocketConnection connection, - XMPPPacketReader reader) throws DocumentException, IOException, XmlPullParserException { - - StringBuilder sb = new StringBuilder(); - sb.append(""); - sb.append(StringUtils.encodeBase64(localDomain)); - sb.append(""); - connection.deliverRawText(sb.toString()); - - Element response = reader.parseDocument().getRootElement(); - return response != null && "success".equals(response.getName()); - } - - public static LocalOutgoingServerSession encryptAndAuthenticate(DomainPair domainPair, SocketConnection connection, XMPPPacketReader reader, StringBuilder openingStream) throws Exception { - final Logger log = LoggerFactory.getLogger(Log.getName() + "[Encrypt connection for: " + domainPair + "]" ); - Element features; - - log.debug( "Encrypting and authenticating connection ..."); - - log.debug( "Indicating we want TLS and wait for response." ); - connection.deliverRawText( "" ); - - MXParser xpp = reader.getXPPParser(); - // Wait for the response - Element proceed = reader.parseDocument().getRootElement(); - if (proceed != null && proceed.getName().equals("proceed")) { - log.debug( "Received 'proceed' from remote server. Negotiating TLS..." ); - try { -// boolean needed = JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_CERTIFICATE_VERIFY, true) && -// JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_CERTIFICATE_CHAIN_VERIFY, true) && -// !JiveGlobals.getBooleanProperty(ConnectionSettings.Server.TLS_ACCEPT_SELFSIGNED_CERTS, false); - connection.startTLS(true, false); - } catch(Exception e) { - log.debug("TLS negotiation failed: " + e.getMessage()); - throw e; - } - log.debug( "TLS negotiation was successful. Connection encrypted. Proceeding with authentication..." ); - - // If TLS cannot be used for authentication, it is permissible to use another authentication mechanism - // such as dialback. RFC 6120 does not explicitly allow this, as it does not take into account any other - // authentication mechanism other than TLS (it does mention dialback in an interoperability note. However, - // RFC 7590 Section 3.4 writes: "In particular for XMPP server-to-server interactions, it can be reasonable - // for XMPP server implementations to accept encrypted but unauthenticated connections when Server Dialback - // keys [XEP-0220] are used." In short: if Dialback is allowed, unauthenticated TLS is better than no TLS. - if (!SASLAuthentication.verifyCertificates(connection.getPeerCertificates(), domainPair.getRemote(), true)) { - if (JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { - log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); - return null; - } - if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { - log.debug( "SASL authentication failed. Will continue with dialback." ); - } else { - log.warn( "Unable to authenticate the connection: SASL authentication failed (and dialback is not available)." ); - return null; - } - } - - log.debug( "TLS negotiation was successful so initiate a new stream." ); - connection.deliverRawText( openingStream.toString() ); - - // Reset the parser to use the new secured reader - xpp.setInput(new InputStreamReader(connection.getTLSStreamHandler().getInputStream(), StandardCharsets.UTF_8)); - // Skip new stream element - for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { - eventType = xpp.next(); - } - // Get the stream ID - String id = xpp.getAttributeValue("", "id"); - // Get new stream features - features = reader.parseDocument().getRootElement(); - if (features != null) { - return authenticate( domainPair, connection, reader, openingStream, features, id ); - } - else { - log.debug( "Failed to encrypt and authenticate connection: neither SASL mechanisms nor SERVER DIALBACK were offered by the remote host." ); + log.warn("Unable to create a new session: Dialback (as a fallback) failed."); return null; } - } - else { - log.debug( "Failed to encrypt and authenticate connection: was not received!" ); - return null; - } - } - - private static LocalOutgoingServerSession authenticate( final DomainPair domainPair, - final SocketConnection connection, - final XMPPPacketReader reader, - final StringBuilder openingStream, - final Element features, - final String id ) throws DocumentException, IOException, XmlPullParserException - { - final Logger log = LoggerFactory.getLogger(Log.getName() + "[Authenticate connection for: " + domainPair + "]" ); - - MXParser xpp = reader.getXPPParser(); - - // Bookkeeping: determine what functionality the remote server offers. - boolean saslEXTERNALoffered = false; - if (features.element("mechanisms") != null) { - Iterator it = features.element( "mechanisms").elementIterator(); - while (it.hasNext()) { - Element mechanism = it.next(); - if ("EXTERNAL".equals(mechanism.getTextTrim())) { - saslEXTERNALoffered = true; - break; - } - } - } - final boolean dialbackOffered = features.element("dialback") != null; - - log.debug("Remote server is offering dialback: {}, EXTERNAL SASL: {}", dialbackOffered, saslEXTERNALoffered ); - - LocalOutgoingServerSession result = null; - - // first, try SASL - if (saslEXTERNALoffered) { - log.debug( "Trying to authenticate with EXTERNAL SASL." ); - result = attemptSASLexternal(connection, xpp, reader, domainPair, id, openingStream); - if (result == null) { - log.debug( "Failed to authenticate with EXTERNAL SASL." ); - } else { - log.debug( "Successfully authenticated with EXTERNAL SASL." ); - } - } - - // SASL unavailable or failed, try dialback. - if (result == null) { - log.debug( "Trying to authenticate with dialback." ); - result = attemptDialbackOverTLS(connection, reader, domainPair, id); - if (result == null) { - log.debug( "Failed to authenticate with dialback." ); - } else { - log.debug( "Successfully authenticated with dialback." ); - } - } - - if ( result != null ) { - log.debug( "Successfully encrypted and authenticated connection!" ); - return result; } else { - log.warn( "Unable to encrypt and authenticate connection: Exhausted all options." ); + log.warn("Unable to create a new session: exhausted all options (not trying dialback as a fallback, as server dialback is disabled by configuration."); return null; } } - private static LocalOutgoingServerSession attemptDialbackOverTLS(Connection connection, XMPPPacketReader reader, DomainPair domainPair, String id) { - final Logger log = LoggerFactory.getLogger( Log.getName() + "[Dialback over TLS for: " + domainPair + " (Stream ID: " + id + ")]" ); - - if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { - log.debug("Trying to connecting using dialback over TLS."); - ServerDialback method = new ServerDialback(connection, domainPair); - OutgoingServerSocketReader newSocketReader = new OutgoingServerSocketReader(reader); - if (method.authenticateDomain(newSocketReader, id)) { - log.debug("Dialback over TLS was successful."); - StreamID streamID = BasicStreamIDFactory.createStreamID(id); - LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, newSocketReader, streamID); - connection.init(session); - // Set the remote domain name as the address of the session. - session.setAddress(new JID(null, domainPair.getRemote(), null)); - session.setAuthenticationMethod(AuthenticationMethod.DIALBACK); - return session; - } - else { - log.debug("Dialback over TLS failed"); - return null; - } - } - else { - log.debug("Skipping server dialback attempt as it has been disabled by local configuration."); - return null; - } - } - - private static LocalOutgoingServerSession attemptSASLexternal(SocketConnection connection, MXParser xpp, XMPPPacketReader reader, DomainPair domainPair, String id, StringBuilder openingStream) throws DocumentException, IOException, XmlPullParserException { - final Logger log = LoggerFactory.getLogger( Log.getName() + "[EXTERNAL SASL for: " + domainPair + " (Stream ID: " + id + ")]" ); - - log.debug("Starting EXTERNAL SASL."); - if (doExternalAuthentication(domainPair.getLocal(), connection, reader)) { - log.debug("EXTERNAL SASL was successful."); - // SASL was successful so initiate a new stream - connection.deliverRawText(openingStream.toString()); - - // Reset the parser - //xpp.resetInput(); - // // Reset the parser to use the new secured reader - xpp.setInput(new InputStreamReader(connection.getTLSStreamHandler().getInputStream(), StandardCharsets.UTF_8)); - // Skip the opening stream sent by the server - for (int eventType = xpp.getEventType(); eventType != XmlPullParser.START_TAG;) { - eventType = xpp.next(); - } - // SASL authentication was successful so create new OutgoingServerSession - id = xpp.getAttributeValue("", "id"); - StreamID streamID = BasicStreamIDFactory.createStreamID(id); - LocalOutgoingServerSession session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, new OutgoingServerSocketReader(reader), streamID); - connection.init(session); - // Set the remote domain name as the address of the session - session.setAddress(new JID(null, domainPair.getRemote(), null)); - // Set that the session was created using TLS+SASL (no server dialback) - session.setAuthenticationMethod(AuthenticationMethod.SASL_EXTERNAL); - return session; - } - else { - log.debug("EXTERNAL SASL failed."); - return null; - } - } - - private static boolean doExternalAuthentication(String localDomain, SocketConnection connection, - XMPPPacketReader reader) throws DocumentException, IOException, XmlPullParserException { - - StringBuilder sb = new StringBuilder(); - sb.append(""); - sb.append(StringUtils.encodeBase64(localDomain)); - sb.append(""); - connection.deliverRawText(sb.toString()); - - Element response = reader.parseDocument().getRootElement(); - return response != null && "success".equals(response.getName()); + private static boolean configDoesNotRequireTls(ConnectionConfiguration listenerConfiguration) { + return listenerConfiguration.getTlsPolicy() != Connection.TLSPolicy.required; } public LocalOutgoingServerSession(String localDomain, Connection connection, OutgoingServerSocketReader socketReader, StreamID streamID) { @@ -864,6 +302,10 @@ public LocalOutgoingServerSession(String localDomain, Connection connection, Out socketReader.setSession(this); } + public LocalOutgoingServerSession(String localDomain, Connection connection, StreamID streamID) { + super(localDomain, connection, streamID); + } + @Override boolean canProcess(Packet packet) { final DomainPair domainPair = new DomainPair(packet.getFrom().getDomain(), packet.getTo().getDomain()); diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java index 947ad9989c..0f0e5957b6 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java @@ -25,6 +25,7 @@ import org.jivesoftware.util.JiveGlobals; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -157,6 +158,14 @@ public void tearDown() throws Exception Fixtures.clearExistingProperties(); } + + @Test + void single() throws Exception { + ServerSettings localServerSettings = new ServerSettings(Connection.TLSPolicy.required, ServerSettings.CertificateState.MISSING, true, true); + ServerSettings remoteServerSettings = new ServerSettings(Connection.TLSPolicy.required, ServerSettings.CertificateState.MISSING, true, true); + outgoingTest(localServerSettings, remoteServerSettings); + } + /** * Unit test in which Openfire initiates an outgoing server-to-server connection. * diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java b/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java index 75f357e5d2..9b541a31e5 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java @@ -50,7 +50,7 @@ public class RemoteReceivingServerDummy extends AbstractRemoteServerDummy implem /** * When switched to 'true', most XMPP interaction will be printed to standard-out. */ - public static final boolean doLog = false; + public static final boolean doLog = true; private ServerSocket server; From 9385d9ea2acaac37fa315c6c1a126bdaddd9464a Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Wed, 21 Jun 2023 13:08:25 +0100 Subject: [PATCH 31/56] OF-2559 Implement Netty inbound connection handlers Replace CopyOnWriteMap from MINA; migrating to netty so replacing MINA utility with something similar. Remove MINA-specific stat collector; migrating to netty so removing MINA specific stat collector. For netty we might look to the following in the future to implement a netty-specific stats collector: - https://netty.io/4.0/api/io/netty/handler/ssl/OpenSslSessionStats.html - https://netty.io/4.0/api/io/netty/handler/traffic/package-summary.html --- .../main/java/org/jivesoftware/openfire/net/SocketUtil.java | 4 ++-- .../jivesoftware/openfire/server/OutgoingSessionPromise.java | 4 ++-- .../openfire/session/LocalIncomingServerSession.java | 1 + 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java index 02abf3ab0a..0abe398a90 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java @@ -69,10 +69,10 @@ public static Map.Entry createSocketToXmppDomain( String xmppDo try { // (re)initialize the socket. - socket = new Socket(); + socket = new Socket(); // AG HERE Log.debug( "Trying to create socket connection to XMPP domain '{}' using remote host: {}:{} (blocks up to {} ms) ...", xmppDomain, realHostname, realPort, socketTimeout ); - socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); + socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); // AG HERE Log.debug( "Successfully created socket connection to XMPP domain '{}' using remote host: {}:{}!", xmppDomain, realHostname, realPort ); return new AbstractMap.SimpleEntry<>(socket, directTLS); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java index b76e2603a7..2496a6386d 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java @@ -278,7 +278,7 @@ public void run() { Log.trace("Finished processing {}", domainPair); } - private RoutableChannelHandler establishConnection() throws Exception { + private RoutableChannelHandler establishConnection() throws Exception { // TODO: AG - Maybe here? Log.debug("Start establishing a connection for {}", domainPair); // Create a connection to the remote server from the domain where the packet has been sent boolean created; @@ -286,7 +286,7 @@ private RoutableChannelHandler establishConnection() throws Exception { final Lock lock = serversCache.getLock(domainPair); lock.lock(); try { - created = LocalOutgoingServerSession.authenticateDomain(domainPair); + created = LocalOutgoingServerSession.authenticateDomain(domainPair); //TODO: AG MAGIC HERE } finally { lock.unlock(); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java index 1bf3958cc4..c5960928a8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java @@ -227,6 +227,7 @@ public static LocalIncomingServerSession createSession(String serverName, XmlPul } } + public LocalIncomingServerSession(String serverName, Connection connection, StreamID streamID, String fromDomain) { super(serverName, connection, streamID); this.fromDomain = fromDomain; From e56b34c6950ee9bb19dcd0da5a3ba31200292105 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 3 Jul 2023 11:59:36 +0100 Subject: [PATCH 32/56] OF-2559 Added TLS handler for inbound netty connections Tested with non-netty outbound server to a netty-based inbound, so using TLS 1.2 as restricted by outbound capabilities until we pull netty through into outbound connection. --- .../spi/EncryptionArtifactFactory.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index dc585cb2a7..e672cc78f9 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -343,6 +343,30 @@ public synchronized SslContextFactory getSslContextFactory() } } + public SslContext createSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { + getKeyManagers(); + SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); + + // Set policy for checking client certificates. + switch ( configuration.getClientAuth() ) + { + case disabled: + builder.clientAuth(ClientAuth.NONE); + break; + case wanted: + builder.clientAuth(ClientAuth.OPTIONAL); + break; + case needed: + builder.clientAuth(ClientAuth.REQUIRE); + break; + } + + builder.protocols(configuration.getEncryptionProtocols()); + builder.startTls(true); + + return builder.build(); + } + /** * Create and configure a new SslContext instance for a Netty server.

* From 545fc8c528caaf9e9653dc191ea7b23941c3348a Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Wed, 5 Jul 2023 14:59:36 +0100 Subject: [PATCH 33/56] OF-2559 S2S outbound with Netty TLS 1.2 & 1.3 working with S2S --- .../openfire/net/StanzaHandler.java | 2 +- .../server/OutgoingSessionPromise.java | 4 ++-- .../spi/EncryptionArtifactFactory.java | 20 ++++++++++++++++++- 3 files changed, 22 insertions(+), 4 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java index 6ac18962ff..ed201b5b5a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java @@ -175,7 +175,7 @@ protected void processStanza(String stanza, XMPPPacketReader reader) throws Exce // only known case occurring 'in the wild' for this is Dialback, but it's valid XML / XMPP regardless). Re- // establishing those prefixes is achieved by wrapping the data-to-be-parsed in a dummy root element on which // the prefixes are defined. After the data has been parsed, the dummy root element is discarded. See OF-2556. - Log.trace("Connection defined namespace prefixes on its original 'stream' element."); + Log.debug("Connection defined namespace prefixes on its original 'stream' element."); final StringBuilder sb = new StringBuilder(); sb.append(" sb.append(" ").append(namespace.asXML())); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java index 2496a6386d..27b22be2b3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java @@ -278,7 +278,7 @@ public void run() { Log.trace("Finished processing {}", domainPair); } - private RoutableChannelHandler establishConnection() throws Exception { // TODO: AG - Maybe here? + private RoutableChannelHandler establishConnection() throws Exception { Log.debug("Start establishing a connection for {}", domainPair); // Create a connection to the remote server from the domain where the packet has been sent boolean created; @@ -286,7 +286,7 @@ private RoutableChannelHandler establishConnection() throws Exception { // TODO: final Lock lock = serversCache.getLock(domainPair); lock.lock(); try { - created = LocalOutgoingServerSession.authenticateDomain(domainPair); //TODO: AG MAGIC HERE + created = LocalOutgoingServerSession.authenticateDomain(domainPair); // TODO: AG MAGIC HERE } finally { lock.unlock(); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index e672cc78f9..549998417a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -343,7 +343,7 @@ public synchronized SslContextFactory getSslContextFactory() } } - public SslContext createSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { + public SslContext createServerModeSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { getKeyManagers(); SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); @@ -362,11 +362,29 @@ public SslContext createSslContext() throws UnrecoverableKeyException, NoSuchAlg } builder.protocols(configuration.getEncryptionProtocols()); + builder.ciphers(configuration.getEncryptionCipherSuites()); builder.startTls(true); return builder.build(); } + public SslContext createClientModeSslContext() throws SSLException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException { + getKeyManagers(); + + // We will never send SSLV2 ClientHello messages + Set protocols = new HashSet<>(configuration.getEncryptionProtocols()); + protocols.remove("SSLv2Hello"); + + return SslContextBuilder + .forClient() + .protocols(protocols) + .ciphers(configuration.getEncryptionCipherSuites()) + .keyManager(keyManagerFactory) + .trustManager(getTrustManagers()[0]) + .startTls(false) + .build(); + } + /** * Create and configure a new SslContext instance for a Netty server.

* From 394c0909308b49d265eeb1bcba36443006e3bbc9 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Thu, 13 Jul 2023 13:09:26 +0100 Subject: [PATCH 34/56] OF-2559 - Direct TLS on Netty inbound and outbound S2S --- .../jivesoftware/openfire/nio/ServerConnectionHandler.java | 0 .../jivesoftware/openfire/spi/EncryptionArtifactFactory.java | 4 ++-- 2 files changed, 2 insertions(+), 2 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java new file mode 100644 index 0000000000..e69de29bb2 diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index 549998417a..4c7d837594 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -343,7 +343,7 @@ public synchronized SslContextFactory getSslContextFactory() } } - public SslContext createServerModeSslContext() throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { + public SslContext createServerModeSslContext(boolean directTLS) throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { getKeyManagers(); SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); @@ -363,7 +363,7 @@ public SslContext createServerModeSslContext() throws UnrecoverableKeyException, builder.protocols(configuration.getEncryptionProtocols()); builder.ciphers(configuration.getEncryptionCipherSuites()); - builder.startTls(true); + builder.startTls(!directTLS); return builder.build(); } From d28136c32afee3d9a45d91d18112fba6516e0ec2 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 17 Jul 2023 14:25:09 +0100 Subject: [PATCH 35/56] OF-2559 Tidy up comments --- .../src/main/java/org/jivesoftware/admin/PluginFilter.java | 6 +++--- .../openfire/net/RespondingServerStanzaHandler.java | 5 ++++- .../main/java/org/jivesoftware/openfire/net/SocketUtil.java | 4 ++-- .../java/org/jivesoftware/openfire/net/StanzaHandler.java | 2 +- .../openfire/server/OutgoingSessionPromise.java | 2 +- 5 files changed, 11 insertions(+), 8 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java index d3168b8186..924bb4ff9c 100644 --- a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java +++ b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java @@ -29,7 +29,7 @@ import java.util.concurrent.ConcurrentHashMap; /** - * A servlet filter that plugin classes can use to dynamically register and un-register filter logic. + * A servlet filter that plugin classes can use to dynamically register and un-register filter logic.

* * This implementation assumes, but does not enforce, that filters installed by plugins are applied to URL patterns that * match the plugin. When filters installed by different plugins are applied to the same URL, the behavior of this @@ -47,10 +47,10 @@ public class PluginFilter implements Filter { /** * Adds a filter to the list of filters that will be run on every request of which the URL matches the URL that * is registered with this filter. More specifically, the request URL should be equal to, or start with, the filter - * URL. + * URL.

* * Multiple filters can be registered on the same URL, in which case they will be executed in the order in which - * they were added. + * they were added.

* * Adding a filter does not initialize the plugin instance. * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index b8acc171c4..23e7b52e84 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -24,7 +24,10 @@ import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.PacketRouter; import org.jivesoftware.openfire.server.ServerDialback; -import org.jivesoftware.openfire.session.*; +import org.jivesoftware.openfire.session.DomainPair; +import org.jivesoftware.openfire.session.LocalOutgoingServerSession; +import org.jivesoftware.openfire.session.LocalSession; +import org.jivesoftware.openfire.session.ServerSession; import org.jivesoftware.openfire.spi.BasicStreamIDFactory; import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.StringUtils; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java index 0abe398a90..02abf3ab0a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketUtil.java @@ -69,10 +69,10 @@ public static Map.Entry createSocketToXmppDomain( String xmppDo try { // (re)initialize the socket. - socket = new Socket(); // AG HERE + socket = new Socket(); Log.debug( "Trying to create socket connection to XMPP domain '{}' using remote host: {}:{} (blocks up to {} ms) ...", xmppDomain, realHostname, realPort, socketTimeout ); - socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); // AG HERE + socket.connect( new InetSocketAddress( realHostname, realPort ), socketTimeout ); Log.debug( "Successfully created socket connection to XMPP domain '{}' using remote host: {}:{}!", xmppDomain, realHostname, realPort ); return new AbstractMap.SimpleEntry<>(socket, directTLS); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java index ed201b5b5a..6ac18962ff 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java @@ -175,7 +175,7 @@ protected void processStanza(String stanza, XMPPPacketReader reader) throws Exce // only known case occurring 'in the wild' for this is Dialback, but it's valid XML / XMPP regardless). Re- // establishing those prefixes is achieved by wrapping the data-to-be-parsed in a dummy root element on which // the prefixes are defined. After the data has been parsed, the dummy root element is discarded. See OF-2556. - Log.debug("Connection defined namespace prefixes on its original 'stream' element."); + Log.trace("Connection defined namespace prefixes on its original 'stream' element."); final StringBuilder sb = new StringBuilder(); sb.append(" sb.append(" ").append(namespace.asXML())); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java index 27b22be2b3..b76e2603a7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/OutgoingSessionPromise.java @@ -286,7 +286,7 @@ private RoutableChannelHandler establishConnection() throws Exception { final Lock lock = serversCache.getLock(domainPair); lock.lock(); try { - created = LocalOutgoingServerSession.authenticateDomain(domainPair); // TODO: AG MAGIC HERE + created = LocalOutgoingServerSession.authenticateDomain(domainPair); } finally { lock.unlock(); } From 2066e829f78224e1d246f3167ecd05368b420c03 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 18 Jul 2023 11:18:01 +0100 Subject: [PATCH 36/56] OF-2559 Improved comments --- .../openfire/spi/EncryptionArtifactFactory.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index 4c7d837594..665c7a029d 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -343,6 +343,12 @@ public synchronized SslContextFactory getSslContextFactory() } } + /** + * Create and configure a new SslContext instance for a Netty server.

+ * + * @param directTLS if the first write request should be encrypted. + * @return A secure socket protocol implementation which acts as a factory for {@link SSLContext} and {@link io.netty.handler.ssl.SslHandler} + */ public SslContext createServerModeSslContext(boolean directTLS) throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { getKeyManagers(); SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); @@ -368,6 +374,13 @@ public SslContext createServerModeSslContext(boolean directTLS) throws Unrecover return builder.build(); } + /** + * Create and configure a new SslContext instance for a Netty client.

+ * + * Used when the Openfire server is acting as a client when making S2S connections. + * + * @return A secure socket protocol implementation which acts as a factory for {@link SSLContext} and {@link io.netty.handler.ssl.SslHandler} + */ public SslContext createClientModeSslContext() throws SSLException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException { getKeyManagers(); From 987eb52be777661c9b8e001f42e43f9aaf78375d Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Fri, 21 Jul 2023 08:21:39 +0100 Subject: [PATCH 37/56] OF-2559 Remove HTML tags in comments --- .../org/jivesoftware/admin/PluginFilter.java | 6 +- .../net/RespondingServerStanzaHandler.java | 5 +- .../spi/EncryptionArtifactFactory.java | 55 ------------------- 3 files changed, 4 insertions(+), 62 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java index 924bb4ff9c..d3168b8186 100644 --- a/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java +++ b/xmppserver/src/main/java/org/jivesoftware/admin/PluginFilter.java @@ -29,7 +29,7 @@ import java.util.concurrent.ConcurrentHashMap; /** - * A servlet filter that plugin classes can use to dynamically register and un-register filter logic.

+ * A servlet filter that plugin classes can use to dynamically register and un-register filter logic. * * This implementation assumes, but does not enforce, that filters installed by plugins are applied to URL patterns that * match the plugin. When filters installed by different plugins are applied to the same URL, the behavior of this @@ -47,10 +47,10 @@ public class PluginFilter implements Filter { /** * Adds a filter to the list of filters that will be run on every request of which the URL matches the URL that * is registered with this filter. More specifically, the request URL should be equal to, or start with, the filter - * URL.

+ * URL. * * Multiple filters can be registered on the same URL, in which case they will be executed in the order in which - * they were added.

+ * they were added. * * Adding a filter does not initialize the plugin instance. * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index 23e7b52e84..b8acc171c4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -24,10 +24,7 @@ import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.PacketRouter; import org.jivesoftware.openfire.server.ServerDialback; -import org.jivesoftware.openfire.session.DomainPair; -import org.jivesoftware.openfire.session.LocalOutgoingServerSession; -import org.jivesoftware.openfire.session.LocalSession; -import org.jivesoftware.openfire.session.ServerSession; +import org.jivesoftware.openfire.session.*; import org.jivesoftware.openfire.spi.BasicStreamIDFactory; import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.StringUtils; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index 665c7a029d..dc585cb2a7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -398,61 +398,6 @@ public SslContext createClientModeSslContext() throws SSLException, Unrecoverabl .build(); } - /** - * Create and configure a new SslContext instance for a Netty server.

- * - * @param directTLS if the first write request should be encrypted. - * @return A secure socket protocol implementation which acts as a factory for {@link SSLContext} and {@link io.netty.handler.ssl.SslHandler} - */ - public SslContext createServerModeSslContext(boolean directTLS) throws UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, SSLException { - getKeyManagers(); - SslContextBuilder builder = SslContextBuilder.forServer(keyManagerFactory); - - // Set policy for checking client certificates. - switch ( configuration.getClientAuth() ) - { - case disabled: - builder.clientAuth(ClientAuth.NONE); - break; - case wanted: - builder.clientAuth(ClientAuth.OPTIONAL); - break; - case needed: - builder.clientAuth(ClientAuth.REQUIRE); - break; - } - - builder.protocols(configuration.getEncryptionProtocols()); - builder.ciphers(configuration.getEncryptionCipherSuites()); - builder.startTls(!directTLS); - - return builder.build(); - } - - /** - * Create and configure a new SslContext instance for a Netty client.

- * - * Used when the Openfire server is acting as a client when making S2S connections. - * - * @return A secure socket protocol implementation which acts as a factory for {@link SSLContext} and {@link io.netty.handler.ssl.SslHandler} - */ - public SslContext createClientModeSslContext() throws SSLException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException { - getKeyManagers(); - - // We will never send SSLV2 ClientHello messages - Set protocols = new HashSet<>(configuration.getEncryptionProtocols()); - protocols.remove("SSLv2Hello"); - - return SslContextBuilder - .forClient() - .protocols(protocols) - .ciphers(configuration.getEncryptionCipherSuites()) - .keyManager(keyManagerFactory) - .trustManager(getTrustManagers()[0]) - .startTls(false) - .build(); - } - /** * Creates an Apache MINA SslFilter that is configured to use server mode when handshaking. * From 3f4b065a6b764a16f2e94b82b4af1d8893c42e89 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Tue, 25 Jul 2023 14:13:34 +0100 Subject: [PATCH 38/56] feat: OF-2599 - add stats updates on Netty channel read and write operations --- .../openfire/nio/NettyConnection.java | 29 ++++++++++ .../openfire/nio/NettyConnectionHandler.java | 56 ++++++++----------- .../openfire/spi/NettyServerInitializer.java | 10 ++-- 3 files changed, 57 insertions(+), 38 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 58cb8d8f3a..4684a3f783 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -23,10 +23,12 @@ import io.netty.handler.codec.compression.JZlibEncoder; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; +import io.netty.handler.traffic.ChannelTrafficShapingHandler; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.ConnectionCloseListener; import org.jivesoftware.openfire.PacketDeliverer; import org.jivesoftware.openfire.auth.UnauthorizedException; +import org.jivesoftware.openfire.net.ServerTrafficCounter; import org.jivesoftware.openfire.net.StanzaHandler; import org.jivesoftware.openfire.session.LocalSession; import org.jivesoftware.openfire.session.Session; @@ -50,6 +52,8 @@ import java.util.concurrent.atomic.AtomicReference; import static com.jcraft.jzlib.JZlib.Z_BEST_COMPRESSION; +import static org.jivesoftware.openfire.nio.NettyConnectionHandler.WRITTEN_BYTES; +import static org.jivesoftware.openfire.spi.NettyServerInitializer.TRAFFIC_HANDLER_NAME; /** * Implementation of {@link Connection} interface specific for Netty connections. @@ -322,6 +326,8 @@ public void deliver(Packet packet) throws UnauthorizedException { boolean errorDelivering = false; try { ChannelFuture f = channelHandlerContext.writeAndFlush(packet.getElement().asXML()); + updateWrittenBytesCounter(channelHandlerContext); + // TODO handle errors? } catch (Exception e) { @@ -350,6 +356,7 @@ public void deliverRawText(String text) { if (!isClosed()) { boolean errorDelivering = false; ChannelFuture f = channelHandlerContext.writeAndFlush(text); + updateWrittenBytesCounter(channelHandlerContext); // TODO handle errors? // try { @@ -369,6 +376,28 @@ public void deliverRawText(String text) { } } + /** + * Updates the system counter of written bytes. This information is used by the outgoing + * bytes statistic. + * + * @param ctx the context for the channel writing bytes + */ + private void updateWrittenBytesCounter(ChannelHandlerContext ctx) { + ChannelTrafficShapingHandler handler = (ChannelTrafficShapingHandler) ctx.channel().pipeline().get(TRAFFIC_HANDLER_NAME); + if (handler != null) { + long currentBytes = handler.trafficCounter().lastWrittenBytes(); + Long prevBytes = ctx.channel().attr(WRITTEN_BYTES).get(); + long delta; + if (prevBytes == null) { + delta = currentBytes; + } else { + delta = currentBytes - prevBytes; + } + ctx.channel().attr(WRITTEN_BYTES).set(currentBytes); + ServerTrafficCounter.incrementOutgoingCounter(delta); + } + } + public void startTLS(boolean clientMode, boolean directTLS) throws Exception { final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index c22bc12b83..3e2e1aca5e 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -19,11 +19,13 @@ import io.netty.channel.ChannelHandler.Sharable; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.traffic.ChannelTrafficShapingHandler; import io.netty.util.AttributeKey; import org.apache.mina.core.session.IoSession; import org.dom4j.io.XMPPPacketReader; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.net.MXParser; +import org.jivesoftware.openfire.net.ServerTrafficCounter; import org.jivesoftware.openfire.net.StanzaHandler; import org.jivesoftware.openfire.spi.ConnectionConfiguration; import org.slf4j.Logger; @@ -32,6 +34,7 @@ import org.xmlpull.v1.XmlPullParserFactory; import org.xmpp.packet.StreamError; +import static org.jivesoftware.openfire.spi.NettyServerInitializer.TRAFFIC_HANDLER_NAME; /** * A NettyConnectionHandler is responsible for creating new sessions, destroying sessions and delivering * received XML stanzas to the proper StanzaHandler.

@@ -48,6 +51,8 @@ public abstract class NettyConnectionHandler extends SimpleChannelInboundHandler private static final Logger Log = LoggerFactory.getLogger(NettyConnectionHandler.class); static final AttributeKey XML_PARSER = AttributeKey.valueOf("XML-PARSER"); public static final AttributeKey CONNECTION = AttributeKey.valueOf("CONNECTION"); + public static final AttributeKey READ_BYTES = AttributeKey.valueOf("READ_BYTES"); + public static final AttributeKey WRITTEN_BYTES = AttributeKey.valueOf("WRITTEN_BYTES"); static final AttributeKey HANDLER = AttributeKey.valueOf("HANDLER"); @@ -107,6 +112,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { // Create a new Connection for the new session final NettyConnection nettyConnection = createNettyConnection(ctx); ctx.channel().attr(CONNECTION).set(nettyConnection); + ctx.channel().attr(READ_BYTES).set(0L); ctx.channel().attr(HANDLER).set(createStanzaHandler(nettyConnection)); } @@ -125,7 +131,7 @@ public void channelRead0(ChannelHandlerContext ctx, String message) { final XMPPPacketReader parser = PARSER_CACHE.get(); // Update counter of read bytes - // updateReadBytesCounter(session); TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- + updateReadBytesCounter(ctx); Log.trace("Handler on " + ctx.channel().localAddress() + " received: " + message); // Let the stanza handler process the received stanza @@ -152,41 +158,23 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { * Updates the system counter of read bytes. This information is used by the incoming * bytes statistic. * - * @param ctx the session that read more bytes from the socket. + * @param ctx the context for the channel reading bytes */ private void updateReadBytesCounter(ChannelHandlerContext ctx) { -// TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- -// long currentBytes = session.getReadBytes(); -// Long prevBytes = (Long) session.getAttribute("_read_bytes"); -// long delta; -// if (prevBytes == null) { -// delta = currentBytes; -// } -// else { -// delta = currentBytes - prevBytes; -// } -// session.setAttribute("_read_bytes", currentBytes); -// ServerTrafficCounter.incrementIncomingCounter(delta); + ChannelTrafficShapingHandler handler = (ChannelTrafficShapingHandler) ctx.channel().pipeline().get(TRAFFIC_HANDLER_NAME); + if (handler != null) { + long currentBytes = handler.trafficCounter().currentReadBytes(); + Long prevBytes = ctx.channel().attr(READ_BYTES).get(); + long delta; + if (prevBytes == null) { + delta = currentBytes; + } + else { + delta = currentBytes - prevBytes; + } + ctx.channel().attr(READ_BYTES).set(currentBytes); + ServerTrafficCounter.incrementIncomingCounter(delta); + } } - /** - * Updates the system counter of written bytes. This information is used by the outgoing - * bytes statistic. - * - * @param session the session that wrote more bytes to the socket. - */ - private void updateWrittenBytesCounter(IoSession session) { -// TODO maybe replace with https://netty.io/4.0/api/io/netty/handler/traffic/TrafficCounter.html#currentReadBytes-- -// long currentBytes = session.getWrittenBytes(); -// Long prevBytes = (Long) session.getAttribute("_written_bytes"); -// long delta; -// if (prevBytes == null) { -// delta = currentBytes; -// } -// else { -// delta = currentBytes - prevBytes; -// } -// session.setAttribute("_written_bytes", currentBytes); -// ServerTrafficCounter.incrementOutgoingCounter(delta); - } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java index c25e45c20d..0e2eae474b 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java @@ -7,10 +7,9 @@ import io.netty.handler.codec.string.StringEncoder; import io.netty.handler.timeout.IdleStateHandler; import io.netty.handler.timeout.WriteTimeoutHandler; -import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; -import org.jivesoftware.openfire.nio.NettyConnectionHandler; -import org.jivesoftware.openfire.nio.NettyIdleStateKeepAliveHandler; -import org.jivesoftware.openfire.nio.NettyXMPPDecoder; +import io.netty.handler.traffic.ChannelTrafficShapingHandler; + +import org.jivesoftware.openfire.nio.*; import org.jivesoftware.util.SystemProperty; import java.time.Duration; @@ -33,6 +32,8 @@ public class NettyServerInitializer extends ChannelInitializer { .setDynamic(true) .build(); + public static final String TRAFFIC_HANDLER_NAME = "trafficShapingHandler" + ; private final NettyConnectionHandler businessLogicHandler; private final boolean directTLS; private final ChannelGroup allChannels; @@ -50,6 +51,7 @@ public void initChannel(SocketChannel ch) throws Exception { int maxIdleTimeBeforePinging = maxIdleTimeBeforeClosing / 2; ch.pipeline() + .addLast(TRAFFIC_HANDLER_NAME, new ChannelTrafficShapingHandler(0)) .addLast(new NettyXMPPDecoder()) .addLast(new StringEncoder()) .addLast("stalledSessionHandler", new WriteTimeoutHandler(Math.toIntExact(WRITE_TIMEOUT_SECONDS.getValue().getSeconds()))) From 20ac715dcf7f3d4ec78789651df52ffc8a9c9efc Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Wed, 26 Jul 2023 08:13:13 +0100 Subject: [PATCH 39/56] OF-2559 Faster fallback to Dialback Rather than wait for the Netty-based session to timeout (default 5s) before attempting dialback auth this commit moves the fallback dialback code into Netty-land by listening for `SslHandshakeCompletionEvent`. There's more refactoring required, I dislike the state leaking through the stanza handler - there is perhaps a need for a connection/session that wraps the netty connection. This concept might already exist but can't quite get my head around it yet. --- .../net/RespondingServerStanzaHandler.java | 13 +++++ .../nio/NettyOutboundConnectionHandler.java | 55 ++++++++++++++++++- .../openfire/nio/NettySessionInitializer.java | 4 +- .../session/LocalOutgoingServerSession.java | 17 +----- 4 files changed, 70 insertions(+), 19 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index b8acc171c4..ccd1023f74 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -44,6 +44,7 @@ public class RespondingServerStanzaHandler extends StanzaHandler { private static final Logger LOG = LoggerFactory.getLogger(RespondingServerStanzaHandler.class); private final DomainPair domainPair; private boolean isSessionAuthenticated = false; + private boolean attemptedAllAuthenticationMethods; /** * Creates a dedicated reader for a socket. @@ -322,4 +323,16 @@ void createSession(String serverName, XmlPullParser xpp, Connection connection) String currentStreamId = xpp.getAttributeValue("", "id"); session = new LocalOutgoingServerSession(domainPair.getLocal(), connection, BasicStreamIDFactory.createStreamID(currentStreamId)); } + + public void setSessionAuthenticated(boolean authenticated) { + this.isSessionAuthenticated = authenticated; + } + + public boolean haveAttemptedAllAuthenticationMethods() { + return attemptedAllAuthenticationMethods; + } + + public void setAttemptedAllAuthenticationMethods(boolean haveAttemptedAllAuthenticationMethods) { + this.attemptedAllAuthenticationMethods = haveAttemptedAllAuthenticationMethods; + } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java index fe0e11925b..93614bbccf 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -17,11 +17,15 @@ package org.jivesoftware.openfire.nio; import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.ssl.SslHandshakeCompletionEvent; +import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.RespondingServerStanzaHandler; import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.server.ServerDialback; import org.jivesoftware.openfire.session.ConnectionSettings; import org.jivesoftware.openfire.session.DomainPair; +import org.jivesoftware.openfire.session.LocalOutgoingServerSession; import org.jivesoftware.openfire.spi.ConnectionConfiguration; import org.jivesoftware.util.JiveGlobals; import org.slf4j.Logger; @@ -39,12 +43,21 @@ public class NettyOutboundConnectionHandler extends NettyConnectionHandler { private static final Logger Log = LoggerFactory.getLogger(NettyOutboundConnectionHandler.class); private final DomainPair domainPair; + private final int port; + volatile boolean sslInitDone; - public NettyOutboundConnectionHandler(ConnectionConfiguration configuration, DomainPair domainPair) { + public NettyOutboundConnectionHandler(ConnectionConfiguration configuration, DomainPair domainPair, int port) { super(configuration); this.domainPair = domainPair; + this.port = port; } + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + if (sslInitDone) { + super.channelActive(ctx); + } + } @Override NettyConnection createNettyConnection(ChannelHandlerContext ctx) { return new NettyConnection(ctx, null, configuration); @@ -83,4 +96,44 @@ public void handlerAdded(ChannelHandlerContext ctx) { Log.trace("Adding NettyOutboundConnectionHandler"); super.handlerAdded(ctx); } + + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (!sslInitDone && evt instanceof SslHandshakeCompletionEvent) { + SslHandshakeCompletionEvent e = (SslHandshakeCompletionEvent) evt; + if (e.isSuccess()) { + sslInitDone = true; + ctx.fireChannelActive(); + } else { + // SSL Handshake has failed, fall back to dialback + RespondingServerStanzaHandler stanzaHandler = (RespondingServerStanzaHandler) ctx.channel().attr(NettyConnectionHandler.HANDLER).get(); + + if (ServerDialback.isEnabled() && connectionConfigDoesNotRequireTls()) { + Log.debug("Unable to create a new session. Going to try connecting using server dialback as a fallback."); + + // Use server dialback (pre XMPP 1.0) over a plain connection + final LocalOutgoingServerSession outgoingSession = new ServerDialback(domainPair).createOutgoingSession(port); + if (outgoingSession != null) { + Log.debug("Successfully created new session (using dialback as a fallback)!"); + stanzaHandler.setSessionAuthenticated(true); + stanzaHandler.setSession(outgoingSession); + } else { + Log.warn("Unable to create a new session: Dialback (as a fallback) failed."); + stanzaHandler.setSession(null); + } + } else { + Log.warn("Unable to create a new session: exhausted all options (not trying dialback as a fallback, as server dialback is disabled by configuration."); + stanzaHandler.setSession(null); + } + + stanzaHandler.setAttemptedAllAuthenticationMethods(true); + } + } + + super.userEventTriggered(ctx, evt); + } + + private boolean connectionConfigDoesNotRequireTls() { + return this.configuration.getTlsPolicy() != Connection.TLSPolicy.required; + } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java index b06b78f3f6..740c4d0782 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -88,7 +88,7 @@ public Future init(ConnectionConfiguration listenerConfiguration) public void initChannel(SocketChannel ch) throws Exception { ch.pipeline().addLast(new NettyXMPPDecoder()); ch.pipeline().addLast(new StringEncoder()); - ch.pipeline().addLast(new NettyOutboundConnectionHandler(listenerConfiguration, domainPair)); + ch.pipeline().addLast(new NettyOutboundConnectionHandler(listenerConfiguration, domainPair, port)); // Should have a connection if (directTLS) { ch.attr(CONNECTION).get().startTLS(true, true); @@ -140,7 +140,7 @@ private Future waitForSession(Channel channel) { RespondingServerStanzaHandler stanzaHandler = (RespondingServerStanzaHandler) channel.attr(NettyConnectionHandler.HANDLER).get(); return executor.submit(() -> { - while (!stanzaHandler.isSessionAuthenticated()) { + while (!stanzaHandler.isSessionAuthenticated() && !stanzaHandler.haveAttemptedAllAuthenticationMethods()) { Thread.sleep(100); } return stanzaHandler.getSession(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index 32c565f460..8647d2fcb2 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -273,22 +273,7 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai sessionInitialiser.stop(); } - if (ServerDialback.isEnabled() && configDoesNotRequireTls(listenerConfiguration)) { - log.debug("Unable to create a new session. Going to try connecting using server dialback as a fallback."); - - // Use server dialback (pre XMPP 1.0) over a plain connection - final LocalOutgoingServerSession outgoingSession = new ServerDialback(domainPair).createOutgoingSession(port); - if (outgoingSession != null) { // TODO this success handler behaves differently from a similar success handler above. Shouldn't those be the same? - log.debug("Successfully created new session (using dialback as a fallback)!"); - return outgoingSession; - } else { - log.warn("Unable to create a new session: Dialback (as a fallback) failed."); - return null; - } - } else { - log.warn("Unable to create a new session: exhausted all options (not trying dialback as a fallback, as server dialback is disabled by configuration."); - return null; - } + return null; } From 7e43ceeb068824891eff251eaf41c001433aa227 Mon Sep 17 00:00:00 2001 From: Guus der Kinderen Date: Wed, 26 Jul 2023 12:19:48 +0200 Subject: [PATCH 40/56] Remove duplication of connection configuration The Connection interface defines methods to read the configuration of TLS and compression policies, even though these are also defined by the ConnectionConfiguration instance that is used to create the connection. It is undesirable to have the configuration of a connection be defined in various places, or be modified after the original connection has been applied. This commit removes the duplication, and ensures that connection configuration is applied as soon as the instance is created. As a side-effect, this solves an issue with the new Netty code, that never explicitly sets the tlsPolicy on the connection. The single functional aspect of the separation of tlsPolicy between connection and configuration (prior to this change) was the following: the state of the connection-tlsPolicy was used to implicitly define if a session was initialized (this was used to close a connection that was sending unencrypted data, when its configuration required encryption). This commit replaces that implicit defintion by a new, explicit 'isInitialized` method on the Connection interface. --- .../org/jivesoftware/openfire/Connection.java | 43 +++------------- .../net/RespondingServerStanzaHandler.java | 4 +- .../openfire/net/SocketConnection.java | 51 ++----------------- .../openfire/net/SocketReader.java | 6 +-- .../openfire/net/SocketReadingMode.java | 6 +-- .../openfire/net/StanzaHandler.java | 7 ++- .../openfire/net/VirtualConnection.java | 27 ++-------- .../openfire/nio/NIOConnection.java | 33 ++---------- .../openfire/nio/NettyConnection.java | 34 ++----------- .../openfire/server/ServerDialback.java | 2 +- .../openfire/session/LocalClientSession.java | 43 ++++++---------- .../LocalConnectionMultiplexerSession.java | 14 ++--- .../session/LocalIncomingServerSession.java | 35 +++++-------- 13 files changed, 71 insertions(+), 234 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java index 2d8e648252..2f8e8cb195 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java @@ -72,6 +72,13 @@ public interface Connection extends Closeable { */ void reinit( LocalSession session ); + /** + * Checks if the connection has finished initialization. + * + * @return true if connection has finished initialization. + */ + boolean isInitialized(); + /** * Returns the raw IP address of this InetAddress * object. The result is in network byte order: the highest order @@ -306,42 +313,6 @@ default boolean isEncrypted() { */ boolean isCompressed(); - /** - * Returns whether compression is optional or is disabled. - * - * @return whether compression is optional or is disabled. - */ - CompressionPolicy getCompressionPolicy(); - - /** - * Sets whether compression is enabled or is disabled. - * - * @param compressionPolicy whether Compression is enabled or is disabled. - */ - void setCompressionPolicy(CompressionPolicy compressionPolicy); - - /** - * Returns whether TLS is mandatory, optional or is disabled. When TLS is mandatory clients - * are required to encrypt their connections or otherwise their connections will be closed. - * On the other hand, when TLS is disabled clients are not allowed to encrypt their connections - * using TLS. Their connections will be closed if they try to encrypt the connection. in this - * last case. - * - * @return whether TLS is mandatory, optional or is disabled. - */ - TLSPolicy getTlsPolicy(); - - /** - * Sets whether TLS is mandatory, optional or is disabled. When TLS is mandatory clients - * are required to encrypt their connections or otherwise their connections will be closed. - * On the other hand, when TLS is disabled clients are not allowed to encrypt their connections - * using TLS. Their connections will be closed if they try to encrypt the connection. in this - * last case. - * - * @param tlsPolicy whether TLS is mandatory, optional or is disabled. - */ - void setTlsPolicy(TLSPolicy tlsPolicy); - /** * Returns the TLS protocol name used by the connection of the session, if any. * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index ccd1023f74..1baf94f603 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -283,11 +283,11 @@ boolean processUnknowPacket(Element doc) { } private boolean shouldUseTls() { - return connection.getTlsPolicy() == Connection.TLSPolicy.optional || connection.getTlsPolicy() == Connection.TLSPolicy.required; + return connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.optional || connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required; } private boolean mustUseTls() { - return connection.getTlsPolicy() == Connection.TLSPolicy.required; + return connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required; } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java index 4348e296a6..a62cc3c6ed 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java @@ -102,17 +102,8 @@ public class SocketConnection implements Connection { private long writeStarted = -1; - /** - * TLS policy currently in use for this connection. - */ - private TLSPolicy tlsPolicy = TLSPolicy.optional; private boolean usingSelfSignedCertificate; - /** - * Compression policy currently in use for this connection. - */ - private CompressionPolicy compressionPolicy = CompressionPolicy.disabled; - public static Collection getInstances() { return instances.keySet(); } @@ -146,9 +137,6 @@ public SocketConnection(PacketDeliverer backupDeliverer, Socket socket, boolean xmlSerializer = new XMLSocketWriter(writer, this); instances.put(this, ""); - - // Default this sensibly. - this.tlsPolicy = this.getConfiguration().getTlsPolicy(); } /** @@ -277,6 +265,11 @@ public void reinit(LocalSession owner) { } } + @Override + public boolean isInitialized() { + return session != null && !isClosed(); + } + @Override public void registerCloseListener(ConnectionCloseListener listener, Object handbackMessage) { if (isClosed()) { @@ -356,25 +349,6 @@ public boolean isCompressed() { return compressed; } - @Override - public TLSPolicy getTlsPolicy() { - return tlsPolicy; - } - - /** - * Sets whether TLS is mandatory, optional or is disabled. When TLS is mandatory clients - * are required to encrypt their connections or otherwise their connections will be closed. - * On the other hand, when TLS is disabled clients are not allowed to encrypt their connections - * using TLS. Their connections will be closed if they try to encrypt the connection. in this - * last case. - * - * @param tlsPolicy whether TLS is mandatory, optional or is disabled. - */ - @Override - public void setTlsPolicy(TLSPolicy tlsPolicy) { - this.tlsPolicy = tlsPolicy; - } - @Override public Optional getTLSProtocolName() { @@ -391,21 +365,6 @@ public Optional getCipherSuiteName() .map(SSLSession::getCipherSuite); } - @Override - public CompressionPolicy getCompressionPolicy() { - return compressionPolicy; - } - - /** - * Sets whether compression is enabled or is disabled. - * - * @param compressionPolicy whether Compression is enabled or is disabled. - */ - @Override - public void setCompressionPolicy(CompressionPolicy compressionPolicy) { - this.compressionPolicy = compressionPolicy; - } - public long getIdleTimeout() { return idleTimeout; } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReader.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReader.java index 1066fd0678..e68483a72c 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReader.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReader.java @@ -260,7 +260,7 @@ else if ("iq".equals(tag)) { */ protected void processIQ(IQ packet) throws UnauthorizedException { // Ensure that connection was encrypted if TLS was required. - if (connection.getTlsPolicy() == Connection.TLSPolicy.required && + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !connection.isEncrypted()) { closeNeverEncryptedConnection(); return; @@ -283,7 +283,7 @@ protected void processIQ(IQ packet) throws UnauthorizedException { */ protected void processPresence(Presence packet) throws UnauthorizedException { // Ensure that connection was encrypted if TLS was required - if (connection.getTlsPolicy() == Connection.TLSPolicy.required && + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !connection.isEncrypted()) { closeNeverEncryptedConnection(); return; @@ -306,7 +306,7 @@ protected void processPresence(Presence packet) throws UnauthorizedException { */ protected void processMessage(Message packet) throws UnauthorizedException { // Ensure that connection was encrypted if TLS was required - if (connection.getTlsPolicy() == Connection.TLSPolicy.required && + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !connection.isEncrypted()) { closeNeverEncryptedConnection(); return; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReadingMode.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReadingMode.java index d99a23172f..300d21cc40 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReadingMode.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketReadingMode.java @@ -64,7 +64,7 @@ protected SocketReadingMode(Socket socket, SocketReader socketReader) { * @return true if the connection was encryped. */ protected boolean negotiateTLS() { - if (socketReader.connection.getTlsPolicy() == Connection.TLSPolicy.disabled) { + if (socketReader.connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.disabled) { // Send a not_authorized error and close the underlying connection socketReader.connection.close(new StreamError(StreamError.Condition.not_authorized, "A request to negotiate TLS is denied, as TLS has been disabled by configuration.")); // Log a warning so that admins can track this case from the server side @@ -117,7 +117,7 @@ protected void tlsNegotiated() throws XmlPullParserException, IOException { protected boolean authenticateClient(Element doc) throws DocumentException, IOException, XmlPullParserException { // Ensure that connection was encrypted if TLS was required - if (socketReader.connection.getTlsPolicy() == Connection.TLSPolicy.required && + if (socketReader.connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !socketReader.connection.isEncrypted()) { socketReader.closeNeverEncryptedConnection(); return false; @@ -177,7 +177,7 @@ protected void saslSuccessful() throws XmlPullParserException, IOException { */ protected boolean compressClient(Element doc) throws IOException, XmlPullParserException { String error = null; - if (socketReader.connection.getCompressionPolicy() == Connection.CompressionPolicy.disabled) { + if (socketReader.connection.getConfiguration().getCompressionPolicy() == Connection.CompressionPolicy.disabled) { // Client requested compression but this feature is disabled error = ""; // Log a warning so that admins can track this case from the server side diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java index 6ac18962ff..27e1e97ec2 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StanzaHandler.java @@ -231,8 +231,7 @@ private void process(Element doc) throws UnauthorizedException { } // Ensure that connection was encrypted if TLS was required - if (connection.getTlsPolicy() == Connection.TLSPolicy.required && - !connection.isEncrypted()) { + if (connection.isInitialized() && connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !connection.isEncrypted()) { closeNeverEncryptedConnection(); return; } @@ -457,7 +456,7 @@ protected void processMessage(Message packet) throws UnauthorizedException { * @return true if the connection was encrypted. */ protected boolean negotiateTLS() { - if (connection.getTlsPolicy() == Connection.TLSPolicy.disabled) { + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.disabled) { // Send a not_authorized error and close the underlying connection connection.close(new StreamError(StreamError.Condition.not_authorized, "A request to negotiate TLS is denied, as TLS has been disabled by configuration.")); // Log a warning so that admins can track this case from the server side @@ -534,7 +533,7 @@ protected void saslSuccessful() { */ protected boolean compressClient(Element doc) { String error = null; - if (connection.getCompressionPolicy() == Connection.CompressionPolicy.disabled) { + if (connection.getConfiguration().getCompressionPolicy() == Connection.CompressionPolicy.disabled) { // Client requested compression but this feature is disabled error = ""; // Log a warning so that admins can track this case from the server side diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java index ef698682b1..4137c6388b 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java @@ -91,18 +91,6 @@ public boolean isClosed() { return state.get() == State.CLOSED; } - @Override - public Connection.CompressionPolicy getCompressionPolicy() { - // Return null since compression is not used for virtual connections - return null; - } - - @Override - public Connection.TLSPolicy getTlsPolicy() { - // Return null since TLS is not used for virtual connections - return null; - } - @Override public boolean isCompressed() { // Return false since compression is not used for virtual connections @@ -114,16 +102,6 @@ public void setXMPPVersion(int majorVersion, int minorVersion) { //Ignore } - @Override - public void setCompressionPolicy(CompressionPolicy compressionPolicy) { - //Ignore - } - - @Override - public void setTlsPolicy(TLSPolicy tlsPolicy) { - //Ignore - } - @Override @Nullable public PacketDeliverer getPacketDeliverer() { @@ -182,6 +160,11 @@ public void reinit(LocalSession session) { } } + @Override + public boolean isInitialized() { + return session != null && !isClosed(); + } + /** * Closes the session, the virtual connection and notifies listeners that the connection * has been closed. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java index 98862a54ec..3d2e897ea8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java @@ -85,16 +85,8 @@ public class NIOConnection implements Connection { private int minorVersion = 0; private String language = null; - /** - * TLS policy currently in use for this connection. - */ - private TLSPolicy tlsPolicy = TLSPolicy.optional; private boolean usingSelfSignedCertificate; - /** - * Compression policy currently in use for this connection. - */ - private CompressionPolicy compressionPolicy = CompressionPolicy.disabled; private static final ThreadLocal encoder = new ThreadLocalEncoder(); /** @@ -307,6 +299,11 @@ public void reinit(LocalSession owner) { } } + @Override + public boolean isInitialized() { + return session != null && !isClosed(); + } + protected StanzaHandler getStanzaHandler() { return (StanzaHandler)ioSession.getAttribute(ConnectionHandler.HANDLER); } @@ -476,26 +473,6 @@ public boolean isCompressed() { return ioSession.getFilterChain().contains(COMPRESSION_FILTER_NAME); } - @Override - public CompressionPolicy getCompressionPolicy() { - return compressionPolicy; - } - - @Override - public void setCompressionPolicy(CompressionPolicy compressionPolicy) { - this.compressionPolicy = compressionPolicy; - } - - @Override - public TLSPolicy getTlsPolicy() { - return tlsPolicy; - } - - @Override - public void setTlsPolicy(TLSPolicy tlsPolicy) { - this.tlsPolicy = tlsPolicy; - } - @Override public String toString() { return super.toString() + " MINA Session: " + ioSession; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 4684a3f783..2bfefe9569 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -85,17 +85,8 @@ public class NettyConnection implements Connection { private int majorVersion = 1; private int minorVersion = 0; - /** - * TLS policy currently in use for this connection. - */ - private TLSPolicy tlsPolicy = TLSPolicy.optional; private boolean usingSelfSignedCertificate; - /** - * Compression policy currently in use for this connection. - */ - private CompressionPolicy compressionPolicy = CompressionPolicy.disabled; - /** * Flag that specifies if the connection should be considered closed. Closing a NIO connection * is an asynch operation so instead of waiting for the connection to be actually closed just @@ -297,6 +288,11 @@ public void reinit(LocalSession owner) { } } + @Override + public boolean isInitialized() { + return session != null && !isClosed(); + } + @Override public boolean isClosed() { return state.get() == State.CLOSED; @@ -458,26 +454,6 @@ public boolean isCompressed() { return channelHandlerContext.channel().pipeline().get(JZlibDecoder.class) != null; } - @Override - public CompressionPolicy getCompressionPolicy() { - return compressionPolicy; - } - - @Override - public void setCompressionPolicy(CompressionPolicy compressionPolicy) { - this.compressionPolicy = compressionPolicy; - } - - @Override - public TLSPolicy getTlsPolicy() { - return tlsPolicy; - } - - @Override - public void setTlsPolicy(TLSPolicy tlsPolicy) { - this.tlsPolicy = tlsPolicy; - } - @Override public String toString() { return super.toString() + " Netty Session: " + channelHandlerContext.name(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java b/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java index abc62ee790..9fc7a8e29b 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/server/ServerDialback.java @@ -531,7 +531,7 @@ public void validateRemoteDomain(Element doc, StreamID streamID) throws StreamEr throw new StreamErrorException(new StreamError(StreamError.Condition.policy_violation, "Dialback has been disabled by configuration.")); } - if (connection.getTlsPolicy() == Connection.TLSPolicy.required && !connection.isEncrypted()) { + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !connection.isEncrypted()) { throw new StreamErrorException(new StreamError(StreamError.Condition.policy_violation, "Local server configuration dictates that Server Dialback can be negotiated only after the connection has been encrypted.")); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java index 8f80ede5f1..9ad64c35d9 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java @@ -29,7 +29,6 @@ import org.jivesoftware.openfire.privacy.PrivacyList; import org.jivesoftware.openfire.privacy.PrivacyListManager; import org.jivesoftware.openfire.roster.RosterManager; -import org.jivesoftware.openfire.spi.ConnectionConfiguration; import org.jivesoftware.openfire.streammanagement.StreamManager; import org.jivesoftware.openfire.user.PresenceEventDispatcher; import org.jivesoftware.openfire.user.UserNotFoundException; @@ -228,32 +227,20 @@ public static LocalClientSession createSession(String serverName, XmlPullParser int minorVersion = version[1]; connection.setXMPPVersion(majorVersion, minorVersion); - final ConnectionConfiguration connectionConfiguration = connection.getConfiguration(); - // Indicate the TLS policy to use for this connection - if (!connection.isEncrypted()) { - boolean hasCertificates = false; - try { - hasCertificates = connectionConfiguration.getIdentityStore().getAllCertificates().size() > 0; - } - catch (Exception e) { - Log.error(e.getMessage(), e); - } - Connection.TLSPolicy tlsPolicy = connectionConfiguration.getTlsPolicy(); - if (Connection.TLSPolicy.required == tlsPolicy && !hasCertificates) { - Log.error("Client session rejected. TLS is required but no certificates " + - "were created."); - return null; - } - // Set default TLS policy - connection.setTlsPolicy(hasCertificates ? tlsPolicy : Connection.TLSPolicy.disabled); - } else { - // Set default TLS policy - connection.setTlsPolicy(Connection.TLSPolicy.disabled); + boolean hasCertificates = false; + try { + hasCertificates = !connection.getConfiguration().getIdentityStore().getAllCertificates().isEmpty(); + } + catch (Exception e) { + Log.error("Unable to load find any content in the identity store. This connection won't be able to support TLS.", e); } - // Indicate the compression policy to use for this connection - connection.setCompressionPolicy( connectionConfiguration.getCompressionPolicy() ); + if (!hasCertificates && connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required) { + Log.error("Client session rejected. TLS is required but no certificates " + + "were created."); + return null; + } // Create a ClientSession for this user. LocalClientSession session = SessionManager.getInstance().createClientSession(connection, language); @@ -287,9 +274,9 @@ public static LocalClientSession createSession(String serverName, XmlPullParser sb = new StringBuilder(490); sb.append(""); - if (connection.getTlsPolicy() != Connection.TLSPolicy.disabled) { + if (connection.getConfiguration().getTlsPolicy() != Connection.TLSPolicy.disabled) { sb.append(""); - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required) { sb.append(""); } sb.append(""); @@ -771,14 +758,14 @@ else if (this.presence.isAvailable()) { public String getAvailableStreamFeatures() { // Offer authenticate and registration only if TLS was not required or if required // then the connection is already encrypted - if (conn.getTlsPolicy() == Connection.TLSPolicy.required && !conn.isEncrypted()) { + if (conn.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !conn.isEncrypted()) { return null; } StringBuilder sb = new StringBuilder(200); // Include Stream Compression Mechanism - if (conn.getCompressionPolicy() != Connection.CompressionPolicy.disabled && + if (conn.getConfiguration().getCompressionPolicy() != Connection.CompressionPolicy.disabled && !conn.isCompressed()) { sb.append( "zlib"); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java index de87385beb..24037b4bd6 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java @@ -105,12 +105,6 @@ public static LocalConnectionMultiplexerSession createSession(String serverName, return null; } - // Indicate the TLS policy to use for this connection - connection.setTlsPolicy( connection.getConfiguration().getTlsPolicy() ); - - // Indicate the compression policy to use for this connection - connection.setCompressionPolicy( connection.getConfiguration().getCompressionPolicy() ); - // Set the connection manager domain to use delivering a packet fails final MultiplexerPacketDeliverer packetDeliverer = (MultiplexerPacketDeliverer) connection.getPacketDeliverer(); if (packetDeliverer != null) { @@ -147,9 +141,9 @@ public static LocalConnectionMultiplexerSession createSession(String serverName, sb = new StringBuilder(490); sb.append(""); - if (connection.getTlsPolicy() != Connection.TLSPolicy.disabled) { + if (connection.getConfiguration().getTlsPolicy() != Connection.TLSPolicy.disabled) { sb.append(""); - if (connection.getTlsPolicy() == Connection.TLSPolicy.required) { + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required) { sb.append(""); } sb.append(""); @@ -179,12 +173,12 @@ public LocalConnectionMultiplexerSession(String serverName, Connection connectio @Override public String getAvailableStreamFeatures() { - if (conn.getTlsPolicy() == Connection.TLSPolicy.required && !conn.isEncrypted()) { + if (conn.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required && !conn.isEncrypted()) { return null; } // Include Stream Compression Mechanism - if (conn.getCompressionPolicy() != Connection.CompressionPolicy.disabled && + if (conn.getConfiguration().getCompressionPolicy() != Connection.CompressionPolicy.disabled && !conn.isCompressed()) { return "zlib"; } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java index c5960928a8..056d98be24 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java @@ -160,28 +160,19 @@ public static LocalIncomingServerSession createSession(String serverName, XmlPul Log.trace("Outbound opening stream: {}", openingStream); connection.deliverRawText(openingStream.toString()); - if (serverVersion[0] >= 1) { - // Remote server is XMPP 1.0 compliant so offer TLS and SASL to establish the connection (and server dialback) - - // Indicate the TLS policy to use for this connection - Connection.TLSPolicy tlsPolicy = connection.getTlsPolicy(); - boolean hasCertificates = false; - try { - hasCertificates = XMPPServer.getInstance().getCertificateStoreManager().getIdentityStore( ConnectionType.SOCKET_S2S ).getStore().size() > 0; - } - catch (Exception e) { - Log.error(e.getMessage(), e); - } - if (Connection.TLSPolicy.required == tlsPolicy && !hasCertificates) { - Log.error("Server session rejected. TLS is required but no certificates " + - "were created."); - return null; - } - connection.setTlsPolicy(hasCertificates ? tlsPolicy : Connection.TLSPolicy.disabled); + boolean hasCertificates = false; + try { + hasCertificates = !connection.getConfiguration().getIdentityStore().getAllCertificates().isEmpty(); + } + catch (Exception e) { + Log.error("Unable to load find any content in the identity store. This connection won't be able to support TLS.", e); } - // Indicate the compression policy to use for this connection - connection.setCompressionPolicy( connection.getConfiguration().getCompressionPolicy() ); + if (!hasCertificates && connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required) { + Log.error("Server session rejected. TLS is required but no certificates " + + "were created."); + return null; + } StringBuilder sb = new StringBuilder(); @@ -190,7 +181,7 @@ public static LocalIncomingServerSession createSession(String serverName, XmlPul sb.append(""); - if (!directTLS && (connection.getTlsPolicy() == Connection.TLSPolicy.required || connection.getTlsPolicy() == Connection.TLSPolicy.optional)) { + if (!directTLS && (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required || connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.optional)) { sb.append(""); if (!ServerDialback.isEnabled()) { Log.debug("Server dialback is disabled so TLS is required"); @@ -410,7 +401,7 @@ public String getAvailableStreamFeatures() { StringBuilder sb = new StringBuilder(); // Include Stream Compression Mechanism - if (conn.getCompressionPolicy() != Connection.CompressionPolicy.disabled && + if (conn.getConfiguration().getCompressionPolicy() != Connection.CompressionPolicy.disabled && !conn.isCompressed()) { sb.append("zlib"); } From b43d30b9eddb766c7f1cd0af05fb2cccc54b12b0 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Wed, 26 Jul 2023 10:22:47 +0100 Subject: [PATCH 41/56] fix: OF-2599 - move NettyConnectionHandler creation to a factory method --- .../nio/NettyClientConnectionHandler.java | 3 +- .../nio/NettyConnectionHandlerFactory.java | 48 +++++++++++++++++++ .../openfire/spi/NettyConnectionAcceptor.java | 23 +-------- 3 files changed, 51 insertions(+), 23 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java index 20e1927ef8..15a189dd3f 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java @@ -40,7 +40,7 @@ public class NettyClientConnectionHandler extends NettyConnectionHandler{ * are discarded. */ public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.client.netty-backup-packet-delivery.enabled") // TODO - rename once MINA-specific is removed and NettyClientConnectionHandler becomes ClientConnectionHandler + .setKey("xmpp.client.netty-backup-packet-delivery.enabled") .setDefaultValue(true) .setDynamic(true) .build(); @@ -51,7 +51,6 @@ public NettyClientConnectionHandler(ConnectionConfiguration configuration) { @Override NettyConnection createNettyConnection(ChannelHandlerContext ctx) { - // TODO - can this be moved up to superclass? appears to be same as Server implementation final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; return new NettyConnection(ctx, backupDeliverer, configuration); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java new file mode 100644 index 0000000000..245fb605db --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java @@ -0,0 +1,48 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.jivesoftware.openfire.nio; + +import org.jivesoftware.openfire.spi.ConnectionConfiguration; + +/** + * Responsible for creating the NettyConnectionHandler for the connection type. + * + * @author Alex Gidman + */ +public class NettyConnectionHandlerFactory { + + /** + * Creates a new NettyConnectionHandler based on the type of connection set in the configuration. + * @param configuration options for how the connection is configured + * @return a new NettyConnectionHandler + */ + public static NettyConnectionHandler createConnectionHandler(ConnectionConfiguration configuration) { + switch (configuration.getType()) { + case SOCKET_S2S: + return new NettyServerConnectionHandler(configuration); + case SOCKET_C2S: + return new NettyClientConnectionHandler(configuration); + // TODO add support for Component & Multiplexer + // case COMPONENT: + // return new NettyComponentConnectionHandler(configuration); + // case CONNECTION_MANAGER: + // return new NettyMultiplexerConnectionHandler(configuration); + default: + throw new IllegalStateException("This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType()); + } + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 262d85b708..94bcd94d95 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -26,9 +26,8 @@ import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.util.concurrent.GlobalEventExecutor; import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; import org.jivesoftware.openfire.nio.NettyConnectionHandler; -import org.jivesoftware.openfire.nio.NettyServerConnectionHandler; +import org.jivesoftware.openfire.nio.NettyConnectionHandlerFactory; import org.jivesoftware.util.JiveGlobals; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,29 +83,11 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { */ public NettyConnectionAcceptor(ConnectionConfiguration configuration) { super(configuration); + this.connectionHandler = NettyConnectionHandlerFactory.createConnectionHandler(configuration); String name = configuration.getType().toString().toLowerCase() + (isDirectTLSConfigured() ? "_ssl" : ""); Log = LoggerFactory.getLogger( NettyConnectionAcceptor.class.getName() + "[" + name + "]" ); - switch (configuration.getType()) { - case SOCKET_S2S: - connectionHandler = new NettyServerConnectionHandler(configuration); - break; - case SOCKET_C2S: - connectionHandler = new NettyClientConnectionHandler(configuration); - break; - default: - throw new IllegalStateException("This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType()); - } - - -// TODO add support for COMPONENT & Multiplexer -// case COMPONENT: -// connectionHandler = new ComponentConnectionHandler( configuration ); -// break; -// case CONNECTION_MANAGER: -// connectionHandler = new MultiplexerConnectionHandler( configuration ); -// break; } From c89095a1b10b6fdc48eeac1cd1ea834ee2b99b12 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Wed, 26 Jul 2023 13:48:32 +0100 Subject: [PATCH 42/56] feat: OF-2599 - add NettyComponentConnectionHandler to handle Component connections --- .../nio/NettyComponentConnectionHandler.java | 49 +++++++++++++++++++ .../nio/NettyConnectionHandlerFactory.java | 6 +-- .../openfire/spi/ConnectionListener.java | 6 +-- .../openfire/spi/NettyConnectionAcceptor.java | 4 +- 4 files changed, 56 insertions(+), 9 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java new file mode 100644 index 0000000000..41239d117a --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java @@ -0,0 +1,49 @@ +package org.jivesoftware.openfire.nio; + +import io.netty.channel.ChannelHandlerContext; +import org.jivesoftware.openfire.PacketDeliverer; +import org.jivesoftware.openfire.XMPPServer; +import org.jivesoftware.openfire.net.ComponentStanzaHandler; +import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.jivesoftware.util.JiveGlobals; +import org.jivesoftware.util.SystemProperty; + +/** + * ConnectionHandler that knows which subclass of {@link StanzaHandler} should + * be created and how to build and configure a {@link NettyConnection}. + * + * @author Alex Gidman + */ +public class NettyComponentConnectionHandler extends NettyConnectionHandler { + + /** + * Enable / disable backup delivery of stanzas to the XMPP server itself when a stanza failed to be delivered on a + * component connection. When disabled, stanzas that can not be delivered on the connection are discarded. + */ + public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) + .setKey("xmpp.component.netty-backup-packet-delivery.enabled") + .setDefaultValue(true) + .setDynamic(true) + .build(); + + public NettyComponentConnectionHandler(ConnectionConfiguration configuration) { + super(configuration); + } + + @Override + NettyConnection createNettyConnection(ChannelHandlerContext ctx) { + final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? XMPPServer.getInstance().getPacketDeliverer() : null; + return new NettyConnection(ctx, backupDeliverer, configuration); + } + + @Override + StanzaHandler createStanzaHandler(NettyConnection connection) { + return new ComponentStanzaHandler(XMPPServer.getInstance().getPacketRouter(), connection); + } + + @Override + public int getMaxIdleTime() { + return JiveGlobals.getIntProperty("xmpp.component.idle", 6 * 60 * 1000) / 1000; + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java index 245fb605db..fcb6b9ae6d 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java @@ -36,9 +36,9 @@ public static NettyConnectionHandler createConnectionHandler(ConnectionConfigura return new NettyServerConnectionHandler(configuration); case SOCKET_C2S: return new NettyClientConnectionHandler(configuration); - // TODO add support for Component & Multiplexer - // case COMPONENT: - // return new NettyComponentConnectionHandler(configuration); + case COMPONENT: + return new NettyComponentConnectionHandler(configuration); + // TODO add support for Multiplexer // case CONNECTION_MANAGER: // return new NettyMultiplexerConnectionHandler(configuration); default: diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java index 613291fb56..822c5a64d2 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java @@ -227,10 +227,10 @@ public synchronized void start() } Log.debug( "Starting..." ); - if ( getType() == ConnectionType.SOCKET_S2S || getType() == ConnectionType.SOCKET_C2S ) { - connectionAcceptor = new NettyConnectionAcceptor( generateConnectionConfiguration() ); + if ( getType() == ConnectionType.CONNECTION_MANAGER ) { + connectionAcceptor = new MINAConnectionAcceptor(generateConnectionConfiguration()); } else { - connectionAcceptor = new MINAConnectionAcceptor( generateConnectionConfiguration() ); + connectionAcceptor = new NettyConnectionAcceptor( generateConnectionConfiguration() ); } connectionAcceptor.start(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 94bcd94d95..25c1935536 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -79,7 +79,7 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { private Channel mainChannel; /** - * Instantiates, but not starts, a new instance. + * Instantiates, but does not start, a new instance. */ public NettyConnectionAcceptor(ConnectionConfiguration configuration) { super(configuration); @@ -87,8 +87,6 @@ public NettyConnectionAcceptor(ConnectionConfiguration configuration) { String name = configuration.getType().toString().toLowerCase() + (isDirectTLSConfigured() ? "_ssl" : ""); Log = LoggerFactory.getLogger( NettyConnectionAcceptor.class.getName() + "[" + name + "]" ); - - } /** From c095508e2ae75946334ad9402520c49d08607daf Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Wed, 26 Jul 2023 14:33:53 +0100 Subject: [PATCH 43/56] feat: OF-2599 - add NettyMultiplexerConnectionHandler to handle Mutiplexer connections We can now deprecate (and remove) all NIO components that were built using the Apache MINA framework --- .../nio/ComponentConnectionHandler.java | 2 + .../openfire/nio/ConnectionHandler.java | 2 + .../nio/MultiplexerConnectionHandler.java | 2 + .../openfire/nio/NIOConnection.java | 2 + .../nio/NettyConnectionHandlerFactory.java | 5 +- .../NettyMultiplexerConnectionHandler.java | 66 +++++++++++++++++++ .../openfire/nio/ServerConnectionHandler.java | 0 .../openfire/nio/XMPPCodecFactory.java | 1 + .../openfire/nio/XMPPDecoder.java | 2 + .../openfire/nio/XMPPEncoder.java | 1 + .../openfire/spi/ConnectionListener.java | 7 +- .../openfire/spi/MINAConnectionAcceptor.java | 2 + 12 files changed, 83 insertions(+), 9 deletions(-) create mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java index a485d409e6..afeb82f891 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java @@ -30,7 +30,9 @@ * be created and how to build and configure a {@link NIOConnection}. * * @author Gaston Dombiak + * @deprecated Use {@link NettyComponentConnectionHandler} instead */ +@Deprecated public class ComponentConnectionHandler extends ConnectionHandler { /** diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java index 92d6db7e79..ea692dd229 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java @@ -38,7 +38,9 @@ * received XML stanzas to the proper StanzaHandler. * * @author Gaston Dombiak + * @deprecated Use {@link NettyConnectionHandler} instead */ +@Deprecated public abstract class ConnectionHandler extends IoHandlerAdapter { private static final Logger Log = LoggerFactory.getLogger(ConnectionHandler.class); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java index 94114bd538..1e558afaa8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java @@ -31,7 +31,9 @@ * be created and how to build and configure a {@link org.jivesoftware.openfire.nio.NIOConnection}. * * @author Gaston Dombiak + * @deprecated Use {@link NettyMultiplexerConnectionHandler} instead */ +@Deprecated public class MultiplexerConnectionHandler extends ConnectionHandler { /** diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java index 3d2e897ea8..9836610943 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java @@ -60,7 +60,9 @@ * * @author Gaston Dombiak * @see Apache MINA + * @deprecated Use {@link NettyConnection} instead */ +@Deprecated public class NIOConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(NIOConnection.class); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java index fcb6b9ae6d..ac04fe79f3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandlerFactory.java @@ -38,9 +38,8 @@ public static NettyConnectionHandler createConnectionHandler(ConnectionConfigura return new NettyClientConnectionHandler(configuration); case COMPONENT: return new NettyComponentConnectionHandler(configuration); - // TODO add support for Multiplexer - // case CONNECTION_MANAGER: - // return new NettyMultiplexerConnectionHandler(configuration); + case CONNECTION_MANAGER: + return new NettyMultiplexerConnectionHandler(configuration); default: throw new IllegalStateException("This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType()); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java new file mode 100644 index 0000000000..c8bd0c2f10 --- /dev/null +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2023 Ignite Realtime Foundation. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.jivesoftware.openfire.nio; + +import io.netty.channel.ChannelHandlerContext; +import org.jivesoftware.openfire.PacketDeliverer; +import org.jivesoftware.openfire.XMPPServer; +import org.jivesoftware.openfire.multiplex.MultiplexerPacketDeliverer; +import org.jivesoftware.openfire.net.MultiplexerStanzaHandler; +import org.jivesoftware.openfire.net.StanzaHandler; +import org.jivesoftware.openfire.spi.ConnectionConfiguration; +import org.jivesoftware.util.JiveGlobals; +import org.jivesoftware.util.SystemProperty; + +/** + * ConnectionHandler that knows which subclass of {@link StanzaHandler} should + * be created and how to build and configure a {@link NettyConnection}. + * + * @author Alex Gidman + */ +public class NettyMultiplexerConnectionHandler extends NettyConnectionHandler { + + /** + * Enable / disable backup delivery of stanzas to other connections in the same connection manager when a stanza + * failed to be delivered on a multiplexer (connection manager) connection. When disabled, stanzas that can not + * be delivered on the connection are discarded. + */ + public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) + .setKey("xmpp.multiplex.netty-backup-packet-delivery.enabled") + .setDefaultValue(true) + .setDynamic(true) + .build(); + public NettyMultiplexerConnectionHandler(ConnectionConfiguration configuration) { + super(configuration); + } + + @Override + NettyConnection createNettyConnection(ChannelHandlerContext ctx) { + final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new MultiplexerPacketDeliverer() : null; + return new NettyConnection(ctx, backupDeliverer, configuration); + } + + @Override + StanzaHandler createStanzaHandler(NettyConnection connection) { + return new MultiplexerStanzaHandler(XMPPServer.getInstance().getPacketRouter(), connection); + } + + @Override + public int getMaxIdleTime() { + return JiveGlobals.getIntProperty("xmpp.multiplex.idle", 5 * 60 * 1000) / 1000; + } +} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ServerConnectionHandler.java deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java index fe7ee95ed4..7ebd28aef8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java @@ -26,6 +26,7 @@ * * @author Gaston Dombiak */ +@Deprecated public class XMPPCodecFactory implements ProtocolCodecFactory { private final XMPPEncoder encoder; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java index c512f1d51e..57b8dff820 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java @@ -31,7 +31,9 @@ * stanzas are then passed to the next filters. * * @author Gaston Dombiak + * @deprecated Use {@link NettyXMPPDecoder} instead */ +@Deprecated public class XMPPDecoder extends CumulativeProtocolDecoder { @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java index 9874f528ee..7ad8452823 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java @@ -28,6 +28,7 @@ * * @author Gaston Dombiak */ +@Deprecated public class XMPPEncoder extends ProtocolEncoderAdapter { @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java index 822c5a64d2..ad6e1f873d 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java @@ -227,12 +227,7 @@ public synchronized void start() } Log.debug( "Starting..." ); - if ( getType() == ConnectionType.CONNECTION_MANAGER ) { - connectionAcceptor = new MINAConnectionAcceptor(generateConnectionConfiguration()); - } else { - connectionAcceptor = new NettyConnectionAcceptor( generateConnectionConfiguration() ); - } - + connectionAcceptor = new NettyConnectionAcceptor(generateConnectionConfiguration()); connectionAcceptor.start(); Log.info( "Started." ); } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java index c351cf001f..8523e7303f 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java @@ -50,7 +50,9 @@ class MINAConnectionAcceptor extends ConnectionAcceptor /** * Instantiates, but not starts, a new instance. + * @deprecated Use {@link NettyConnectionAcceptor} instead */ + @Deprecated public MINAConnectionAcceptor( ConnectionConfiguration configuration ) { super( configuration ); From b25f6bab518879d25548f3e7ddeb7c11c88f5958 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Thu, 27 Jul 2023 09:34:34 +0100 Subject: [PATCH 44/56] OF-2559 Ensure NioEventLoopGroup is closed for outbound S2S We were seeing resource limit issues (too many open files) when running Outgoing S2S tests. This was caused by the outbound session initialisation failing to clean up its NioEventLoopGroup in many scenarios. --- .../openfire/nio/NettySessionInitializer.java | 5 ++++- .../openfire/session/LocalOutgoingServerSession.java | 12 +++--------- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java index 740c4d0782..2a89d49486 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettySessionInitializer.java @@ -54,7 +54,7 @@ public class NettySessionInitializer { private final DomainPair domainPair; private final int port; private boolean directTLS = false; - private EventLoopGroup workerGroup; + private final EventLoopGroup workerGroup; private final ExecutorService executor = Executors.newSingleThreadExecutor(); private Channel channel; @@ -118,6 +118,9 @@ public boolean exceptionOccurredForDirectTLS(Throwable cause) { this.channel = b.connect(socketAddress).sync().channel(); + // Make sure we free up resources (worker group NioEventLoopGroup) when the channel is closed + this.channel.closeFuture().addListener(future -> stop()); + // Start the session negotiation sendOpeningStreamHeader(channel); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index 8647d2fcb2..c02db66801 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -260,16 +260,10 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai // Wait for the future to give us a session... // Set a read timeout so that we don't keep waiting forever return (LocalOutgoingServerSession) sessionInitialiser.init(listenerConfiguration).get(INITIALISE_TIMEOUT_SECONDS.getValue().getSeconds(), TimeUnit.SECONDS); - } catch (ExecutionException e) { + } catch (Exception e) { // This might be RFC6120, section 5.4.2.2 "Failure Case" or even an unrelated problem. Handle 'normally'. - log.warn("An exception occurred while creating an encrypted session. Closing connection.", e); - sessionInitialiser.stop(); - return null; - } catch (TimeoutException e) { - log.warn("Timed out waiting for session creation. Closing connection.", e); - sessionInitialiser.stop(); - } catch (InterruptedException e) { - log.warn("An exception occurred while creating an encrypted session. Closing connection.", e); + log.warn("An exception occurred while creating a session. Closing connection.", e); + } finally { sessionInitialiser.stop(); } From ce54590a01608d6d72c8ab94662434673c975a75 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Thu, 27 Jul 2023 12:07:10 +0100 Subject: [PATCH 45/56] feat: OF-2599 - Delete all MINA dependencies, references, and deprecated implementations --- xmppserver/pom.xml | 25 +- .../jivesoftware/openfire/SessionManager.java | 4 +- .../multiplex/MultiplexerPacketDeliverer.java | 6 +- .../openfire/net/ComponentStanzaHandler.java | 4 +- .../openfire/net/StalledSessionsFilter.java | 63 --- .../openfire/net/StartTlsFilter.java | 72 --- .../openfire/nio/ClientConnectionHandler.java | 136 ----- .../nio/ComponentConnectionHandler.java | 67 --- .../openfire/nio/ConnectionHandler.java | 247 --------- .../nio/MultiplexerConnectionHandler.java | 69 --- .../openfire/nio/NIOConnection.java | 492 ------------------ .../nio/NettyClientConnectionHandler.java | 2 +- .../nio/NettyComponentConnectionHandler.java | 2 +- .../openfire/nio/NettyConnectionHandler.java | 1 - .../NettyMultiplexerConnectionHandler.java | 2 +- .../nio/NettyServerConnectionHandler.java | 2 +- .../openfire/nio/OfflinePacketDeliverer.java | 2 +- .../openfire/nio/XMPPCodecFactory.java | 49 -- .../openfire/nio/XMPPDecoder.java | 73 --- .../openfire/nio/XMPPEncoder.java | 39 -- .../openfire/spi/ConnectionListener.java | 19 +- .../openfire/spi/ConnectionManagerImpl.java | 20 - .../spi/EncryptionArtifactFactory.java | 75 --- .../spi/LegacyConnectionAcceptor.java | 149 ------ .../openfire/spi/MINAConnectionAcceptor.java | 288 ---------- .../WebSocketClientConnectionHandler.java | 6 +- .../src/main/webapp/session-details.jsp | 4 +- .../nio/XMLLightweightParserTest.java | 50 +- 28 files changed, 48 insertions(+), 1920 deletions(-) delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/net/StalledSessionsFilter.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/net/StartTlsFilter.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/spi/LegacyConnectionAcceptor.java delete mode 100644 xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java diff --git a/xmppserver/pom.xml b/xmppserver/pom.xml index 611f611dfb..489a399278 100644 --- a/xmppserver/pom.xml +++ b/xmppserver/pom.xml @@ -285,23 +285,6 @@ ${standard-taglib.version} - - - org.apache.mina - mina-core - ${mina.version} - - - org.apache.mina - mina-integration-jmx - ${mina.version} - - - org.apache.mina - mina-filter-compression - ${mina.version} - - io.netty @@ -309,6 +292,14 @@ ${netty.version} + + + com.jcraft + jzlib + 1.1.3 + + + org.bouncycastle diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/SessionManager.java b/xmppserver/src/main/java/org/jivesoftware/openfire/SessionManager.java index 567b812c05..bcfb6af9a1 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/SessionManager.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/SessionManager.java @@ -29,7 +29,7 @@ import org.jivesoftware.openfire.http.HttpConnection; import org.jivesoftware.openfire.http.HttpSession; import org.jivesoftware.openfire.multiplex.ConnectionMultiplexerManager; -import org.jivesoftware.openfire.nio.ClientConnectionHandler; +import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; import org.jivesoftware.openfire.nio.OfflinePacketDeliverer; import org.jivesoftware.openfire.server.OutgoingSessionPromise; import org.jivesoftware.openfire.session.*; @@ -466,7 +466,7 @@ public HttpSession createClientHttpSession(StreamID id, HttpConnection connectio throw new UnauthorizedException("Server not initialized"); } - final PacketDeliverer backupDeliverer = ClientConnectionHandler.BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; + final PacketDeliverer backupDeliverer = NettyClientConnectionHandler.BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; final HttpSession.HttpVirtualConnection vConnection = new HttpSession.HttpVirtualConnection(connection.getRemoteAddr(), backupDeliverer, ConnectionType.SOCKET_C2S); final HttpSession session = new HttpSession(vConnection, serverName, id, connection.getRequestId(), connection.getPeerCertificates(), language, wait, hold, isEncrypted, maxPollingInterval, maxRequests, maxPause, defaultInactivityTimeout, majorVersion, minorVersion); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/multiplex/MultiplexerPacketDeliverer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/multiplex/MultiplexerPacketDeliverer.java index f2d56355ce..b77e91d458 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/multiplex/MultiplexerPacketDeliverer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/multiplex/MultiplexerPacketDeliverer.java @@ -22,7 +22,7 @@ import org.jivesoftware.openfire.PacketException; import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.auth.UnauthorizedException; -import org.jivesoftware.openfire.nio.ClientConnectionHandler; +import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; import org.jivesoftware.openfire.session.ConnectionMultiplexerSession; import org.jivesoftware.util.LocaleUtils; import org.slf4j.Logger; @@ -33,7 +33,7 @@ import org.xmpp.packet.Presence; /** - * Fallback method used by {@link org.jivesoftware.openfire.nio.NIOConnection} when + * Fallback method used by {@link org.jivesoftware.openfire.nio.NettyConnection} when * connected to a connection manager. The fallback method will be used when a connection * fails to send a {@link Packet} (probably because the socket was closed).

* @@ -85,7 +85,7 @@ public void deliver(Packet packet) throws UnauthorizedException, PacketException } private void handleUnprocessedPacket(Packet packet) { - if (!ClientConnectionHandler.BACKUP_PACKET_DELIVERY_ENABLED.getValue()) { + if (!NettyClientConnectionHandler.BACKUP_PACKET_DELIVERY_ENABLED.getValue()) { Log.trace("Discarding packet that was due to be delivered on closed connection {}, for which no other multiplex connections are available, and no client backup deliverer was configured.", this); return; } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ComponentStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ComponentStanzaHandler.java index 3f9bb08a6f..8669596d26 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/ComponentStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/ComponentStanzaHandler.java @@ -38,9 +38,7 @@ * Handler of XML stanzas sent by external components connected directly to the server. Received packet will * have their FROM attribute overridden to avoid spoofing.

* - * This is an implementation of the XEP-114. In the future we will add support for XEP-225 now that - * we are using MINA things should be easier. Since we are now using MINA incoming traffic is handled - * by a set of worker threads. + * This is an implementation of the XEP-114. * * @author Gaston Dombiak */ diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StalledSessionsFilter.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StalledSessionsFilter.java deleted file mode 100644 index a1ae8248ef..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StalledSessionsFilter.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.net; - -import java.io.IOException; -import java.util.Date; - -import org.apache.mina.core.filterchain.IoFilterAdapter; -import org.apache.mina.core.session.IoSession; -import org.apache.mina.core.write.WriteRequest; -import org.jivesoftware.util.JiveGlobals; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * MINA filter that will close sessions that are failing to read outgoing traffic - * and whose outgoing queue is around 5MB. Use the system property {@code session.stalled.cap} - * to set the max number of bytes allowed in the outgoing queue of a session before considering - * it stalled. - * - * @author Gaston Dombiak - */ -public class StalledSessionsFilter extends IoFilterAdapter { - - private static final Logger Log = LoggerFactory.getLogger(StalledSessionsFilter.class); - - private static final int bytesCap = JiveGlobals.getIntProperty("session.stalled.cap", 5242880); - - @Override - public void filterWrite(NextFilter nextFilter, IoSession session, WriteRequest writeRequest) - throws Exception { - if (!session.isClosing()) { - // Get number of pending requests - long pendingBytes = session.getScheduledWriteBytes(); - if (pendingBytes > bytesCap) { - // Get last time we were able to send something to the connected client - long writeTime = session.getLastWriteTime(); - int pendingRequests = session.getScheduledWriteMessages(); - Log.debug("About to kill session with pendingBytes: " + pendingBytes + " pendingWrites: " + - pendingRequests + " lastWrite: " + new Date(writeTime) + "session: " + session); - // Close the session and throw an exception - session.close(false); - throw new IOException("Closing session that seems to be stalled. Preventing OOM"); - } - } - // Call next filter (everything is fine) - super.filterWrite(nextFilter, session, writeRequest); - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StartTlsFilter.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/StartTlsFilter.java deleted file mode 100644 index 93c81144e2..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/StartTlsFilter.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Copyright (C) 2022 Ignite Realtime Foundation. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.net; - -import org.apache.mina.core.buffer.IoBuffer; -import org.apache.mina.core.filterchain.IoFilter; -import org.apache.mina.core.filterchain.IoFilterAdapter; -import org.apache.mina.core.filterchain.IoFilterChain; -import org.apache.mina.core.session.IoSession; -import org.apache.mina.core.write.WriteRequest; -import org.apache.mina.filter.ssl.SslFilter; - -/** - * A MINA filter that facilitates StartTLS negotiation over XMPP. - * - * When StartTLS is being negotiated, an instance of {@link SslFilter} is added to the MINA filter-chain. However, the - * peer also needs to be informed that TLS negotiation can commence, as per section 5.4.2.3 of RFC-6120. - * - * To prevent race conditions, the SSL filter should be set before the peer is told to proceed. However, telling the - * peer to proceed with TLS negotiation needs to happen in plain text, which won't happen if an SSL filter has been set. - * - * As a work-around to this chicken/egg problem, this filter evaluates the data that is sent to the peer prior to it - * being passed to the SSL filter. If the data exactly matches the XMPP 'proceed' element (used to instruct the peer to - * begin TLS negotiation), that data skips the SSL filter. - * - * Instances of this filter are best removed from the filter-chain when TLS has been negotiated, to prevent it from - * evaluating all data that is being sent (which would waste system resources). - * - * @author Guus der Kinderen, guus.der.kinderen@gmail.com - * @see Section 5.4.2.3 of RFC-6120 - */ -public class StartTlsFilter extends IoFilterAdapter -{ - @Override - public void filterWrite(final NextFilter nextFilter, final IoSession session, final WriteRequest writeRequest) - { - final boolean isStartTlsProceed = writeRequest.getOriginalMessage() instanceof IoBuffer && - ((IoBuffer) writeRequest.getOriginalMessage()).hasArray() && - new String(((IoBuffer) writeRequest.getOriginalMessage()).array()).trim().equals(""); - - if (isStartTlsProceed) - { - // This writeRequest is an instruction to the peer for it to start TLS negotiation. This should be sent as - // non-encrypted / plain text data. Skip the SSL filter. - final IoFilterChain chain = session.getFilterChain(); - - for (final IoFilterChain.Entry entry : chain.getAll()) { - IoFilter filter = entry.getFilter(); - - if (filter instanceof SslFilter) { - entry.getNextFilter().filterWrite(session, writeRequest); - } - } - } else { - nextFilter.filterWrite(session, writeRequest); - } - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java deleted file mode 100644 index a337217c01..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ClientConnectionHandler.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.session.IdleStatus; -import org.apache.mina.core.session.IoSession; -import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.PacketDeliverer; -import org.jivesoftware.openfire.SessionManager; -import org.jivesoftware.openfire.XMPPServer; -import org.jivesoftware.openfire.handler.IQPingHandler; -import org.jivesoftware.openfire.net.ClientStanzaHandler; -import org.jivesoftware.openfire.net.StanzaHandler; -import org.jivesoftware.openfire.session.ConnectionSettings; -import org.jivesoftware.openfire.session.LocalClientSession; -import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.util.SystemProperty; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.xmpp.packet.IQ; -import org.xmpp.packet.IQ.Type; -import org.xmpp.packet.JID; - -/** - * ConnectionHandler that knows which subclass of {@link StanzaHandler} should - * be created and how to build and configure a {@link NIOConnection}. - * - * @author Gaston Dombiak - * @deprecated Use {@link NettyClientConnectionHandler} instead - */ -@Deprecated -public class ClientConnectionHandler extends ConnectionHandler { - - private static final Logger Log = LoggerFactory.getLogger(ClientConnectionHandler.class); - - /** - * Enable / disable backup delivery of stanzas to the 'offline message store' of the corresponding user when a stanza - * failed to be delivered on a client connection. When disabled, stanzas that can not be delivered on the connection - * are discarded. - */ - public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.client.backup-packet-delivery.enabled") - .setDefaultValue(true) - .setDynamic(true) - .build(); - - public ClientConnectionHandler(ConnectionConfiguration configuration) { - super(configuration); - } - - @Override - NIOConnection createNIOConnection(IoSession session) { - final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; - return new NIOConnection(session, backupDeliverer, configuration); - } - - @Override - StanzaHandler createStanzaHandler(NIOConnection connection) { - return new ClientStanzaHandler(XMPPServer.getInstance().getPacketRouter(), connection); - } - - @Override - int getMaxIdleTime() { - return (int) ConnectionSettings.Client.IDLE_TIMEOUT_PROPERTY.getValue().toSeconds(); - } - - /** - * In addition to the functionality provided by the parent class, this - * method will send XMPP ping requests to the remote entity on every first - * invocation of this method (which will occur after a period of half the - * allowed connection idle time has passed, without any IO). - * - * XMPP entities must respond with either an IQ result or an IQ error - * (feature-unavailable) stanza upon receiving the XMPP ping stanza. Both - * responses will be received by Openfire and will cause the connection idle - * count to be reset. - * - * Entities that do not respond to the IQ Ping stanzas can be considered - * dead, and their connection will be closed by the parent class - * implementation on the second invocation of this method. - * - * Note that whitespace pings that are sent by XMPP entities will also cause - * the connection idle count to be reset. - * - * @see ConnectionHandler#sessionIdle(IoSession, IdleStatus) - */ - @Override - public void sessionIdle(IoSession session, IdleStatus status) throws Exception { - super.sessionIdle(session, status); - - final boolean doPing = ConnectionSettings.Client.KEEP_ALIVE_PING_PROPERTY.getValue(); - if (doPing && session.getIdleCount(status) == 1) { - final ClientStanzaHandler handler = (ClientStanzaHandler) session.getAttribute(HANDLER); - final JID entity = handler.getAddress(); - - if (entity != null) { - // Ping the connection to see if it is alive. - final IQ pingRequest = new IQ(Type.get); - pingRequest.setChildElement("ping", - IQPingHandler.NAMESPACE); - pingRequest.setFrom( XMPPServer.getInstance().getServerInfo().getXMPPDomain() ); - pingRequest.setTo(entity); - - // Get the connection for this session - final Connection connection = (Connection) session.getAttribute(CONNECTION); - - if (Log.isDebugEnabled()) { - Log.debug("ConnectionHandler: Pinging connection that has been idle: " + connection); - } - - // OF-1497: Ensure that data sent to the client is processed through LocalClientSession, to avoid - // synchronisation issues with stanza counts related to Stream Management (XEP-0198)! - LocalClientSession ofSession = (LocalClientSession) SessionManager.getInstance().getSession( entity ); - if (ofSession == null) { - Log.warn( "Trying to ping a MINA connection that's idle, but has no corresponding Openfire session. MINA Connection: " + connection ); - } else { - ofSession.deliver( pingRequest ); - } - } - } - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java deleted file mode 100644 index afeb82f891..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ComponentConnectionHandler.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.session.IoSession; -import org.jivesoftware.openfire.PacketDeliverer; -import org.jivesoftware.openfire.XMPPServer; -import org.jivesoftware.openfire.net.ComponentStanzaHandler; -import org.jivesoftware.openfire.net.StanzaHandler; -import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.util.JiveGlobals; -import org.jivesoftware.util.SystemProperty; - -/** - * ConnectionHandler that knows which subclass of {@link StanzaHandler} should - * be created and how to build and configure a {@link NIOConnection}. - * - * @author Gaston Dombiak - * @deprecated Use {@link NettyComponentConnectionHandler} instead - */ -@Deprecated -public class ComponentConnectionHandler extends ConnectionHandler { - - /** - * Enable / disable backup delivery of stanzas to the XMPP server itself when a stanza failed to be delivered on a - * component connection. When disabled, stanzas that can not be delivered on the connection are discarded. - */ - public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.component.backup-packet-delivery.enabled") - .setDefaultValue(true) - .setDynamic(true) - .build(); - - public ComponentConnectionHandler(ConnectionConfiguration configuration) { - super(configuration); - } - - @Override - NIOConnection createNIOConnection(IoSession session) { - final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? XMPPServer.getInstance().getPacketDeliverer() : null; - return new NIOConnection(session, backupDeliverer, configuration); - } - - @Override - StanzaHandler createStanzaHandler(NIOConnection connection) { - return new ComponentStanzaHandler(XMPPServer.getInstance().getPacketRouter(), connection); - } - - @Override - int getMaxIdleTime() { - return JiveGlobals.getIntProperty("xmpp.component.idle", 6 * 60 * 1000) / 1000; - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java deleted file mode 100644 index ea692dd229..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/ConnectionHandler.java +++ /dev/null @@ -1,247 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software, 2022-2023 Ignite Realtime Foundation. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.service.IoHandlerAdapter; -import org.apache.mina.core.session.IdleStatus; -import org.apache.mina.core.session.IoSession; -import org.apache.mina.core.write.WriteException; -import org.dom4j.io.XMPPPacketReader; -import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.net.MXParser; -import org.jivesoftware.openfire.net.ServerTrafficCounter; -import org.jivesoftware.openfire.net.StanzaHandler; -import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.util.JiveGlobals; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.xmlpull.v1.XmlPullParserException; -import org.xmlpull.v1.XmlPullParserFactory; -import org.xmpp.packet.StreamError; - -/** - * A ConnectionHandler is responsible for creating new sessions, destroying sessions and delivering - * received XML stanzas to the proper StanzaHandler. - * - * @author Gaston Dombiak - * @deprecated Use {@link NettyConnectionHandler} instead - */ -@Deprecated -public abstract class ConnectionHandler extends IoHandlerAdapter { - - private static final Logger Log = LoggerFactory.getLogger(ConnectionHandler.class); - - static final String XML_PARSER = "XML-PARSER"; - static final String HANDLER = "HANDLER"; - static final String CONNECTION = "CONNECTION"; - - private static final ThreadLocal PARSER_CACHE = new ThreadLocal() - { - @Override - protected XMPPPacketReader initialValue() - { - final XMPPPacketReader parser = new XMPPPacketReader(); - parser.setXPPFactory( factory ); - return parser; - } - }; - /** - * Reuse the same factory for all the connections. - */ - private static XmlPullParserFactory factory = null; - - static { - try { - factory = XmlPullParserFactory.newInstance(MXParser.class.getName(), null); - factory.setNamespaceAware(true); - } - catch (XmlPullParserException e) { - Log.error("Error creating a parser factory", e); - } - } - - /** - * The configuration for new connections. - */ - protected final ConnectionConfiguration configuration; - - protected ConnectionHandler( ConnectionConfiguration configuration ) { - this.configuration = configuration; - } - - @Override - public void sessionOpened(IoSession session) throws Exception { - // Create a new XML parser for the new connection. The parser will be used by the XMPPDecoder filter. - final XMLLightweightParser parser = new XMLLightweightParser(); - session.setAttribute(XML_PARSER, parser); - // Create a new NIOConnection for the new session - final NIOConnection connection = createNIOConnection(session); - session.setAttribute(CONNECTION, connection); - session.setAttribute(HANDLER, createStanzaHandler(connection)); - // Set the max time a connection can be idle before closing it. This amount of seconds - // is divided in two, as Openfire will ping idle clients first (at 50% of the max idle time) - // before disconnecting them (at 100% of the max idle time). This prevents Openfire from - // removing connections without warning. - final int idleTime = getMaxIdleTime() / 2; - if (idleTime > 0) { - session.getConfig().setIdleTime(IdleStatus.READER_IDLE, idleTime); - } - } - - @Override - public void sessionClosed(IoSession session) throws Exception { - final Connection connection = (Connection) session.getAttribute(CONNECTION); - if ( connection != null ) { - connection.close(); - } - } - - /** - * Invoked when a MINA session has been idle for half of the allowed XMPP - * session idle time as specified by {@link #getMaxIdleTime()}. This method - * will be invoked each time that such a period passes (even if no IO has - * occurred in between). - * - * Openfire will disconnect a session the second time this method is - * invoked, if no IO has occurred between the first and second invocation. - * This allows extensions of this class to use the first invocation to check - * for livelyness of the MINA session (e.g by polling the remote entity, as - * {@link ClientConnectionHandler} does). - * - * @see IoHandlerAdapter#sessionIdle(IoSession, IdleStatus) - */ - @Override - public void sessionIdle(IoSession session, IdleStatus status) throws Exception { - if (session.getIdleCount(status) > 1) { - // Get the connection for this session - final Connection connection = (Connection) session.getAttribute(CONNECTION); - if (connection != null) { - // Close idle connection - if (Log.isDebugEnabled()) { - Log.debug("ConnectionHandler: Closing connection that has been idle: {}", connection); - } - connection.close(new StreamError(StreamError.Condition.connection_timeout, "Closing connection due to inactivity.")); - } - } - } - - @Override - public void exceptionCaught(IoSession session, Throwable cause) throws Exception { - Log.warn("Closing connection due to exception in session: {}", session, cause); - - // OF-524: Determine stream:error message. - final StreamError error; - - // OF-1784: Don't write an error when the source problem is an issue with writing data. - if (!JiveGlobals.getBooleanProperty("xmpp.skip-error-delivery-on-write-error.disable", false) && cause instanceof WriteException) { - error = null; - } else if ( cause != null && (cause instanceof XMLNotWellFormedException || (cause.getCause() != null && cause.getCause() instanceof XMLNotWellFormedException) ) ) { - error = new StreamError( StreamError.Condition.not_well_formed ); - } else { - error = new StreamError(StreamError.Condition.internal_server_error); - } - - final Connection connection = (Connection) session.getAttribute(CONNECTION); - if (connection != null) { - connection.close(error); - } - } - - @Override - public void messageReceived(IoSession session, Object message) throws Exception { - // Get the stanza handler for this session - StanzaHandler handler = (StanzaHandler) session.getAttribute(HANDLER); - // Get the parser to use to process stanza. For optimization there is going - // to be a parser for each running thread. Each Filter will be executed - // by the Executor placed as the first Filter. So we can have a parser associated - // to each Thread - final XMPPPacketReader parser = PARSER_CACHE.get(); - // Update counter of read btyes - updateReadBytesCounter(session); - //System.out.println("RCVD: " + message); - // Let the stanza handler process the received stanza - try { - handler.process((String) message, parser); - } catch (Throwable e) { // Make sure to catch Throwable, not (only) Exception! See OF-2367 - Log.error("Closing connection due to error while processing message: {}", message, e); - final Connection connection = (Connection) session.getAttribute(CONNECTION); - if ( connection != null ) { - connection.close(new StreamError(StreamError.Condition.internal_server_error, "An error occurred while processing data raw inbound data.")); - } - } - } - - @Override - public void messageSent(IoSession session, Object message) throws Exception { - super.messageSent(session, message); - // Update counter of written btyes - updateWrittenBytesCounter(session); - //System.out.println("SENT: " + Charset.forName("UTF-8").decode(((ByteBuffer)message).buf())); - } - - abstract NIOConnection createNIOConnection(IoSession session); - - abstract StanzaHandler createStanzaHandler(NIOConnection connection); - - /** - * Returns the max number of seconds a connection can be idle (both ways) before - * being closed.

- * - * @return the max number of seconds a connection can be idle. - */ - abstract int getMaxIdleTime(); - - /** - * Updates the system counter of read bytes. This information is used by the incoming - * bytes statistic. - * - * @param session the session that read more bytes from the socket. - */ - private void updateReadBytesCounter(IoSession session) { - long currentBytes = session.getReadBytes(); - Long prevBytes = (Long) session.getAttribute("_read_bytes"); - long delta; - if (prevBytes == null) { - delta = currentBytes; - } - else { - delta = currentBytes - prevBytes; - } - session.setAttribute("_read_bytes", currentBytes); - ServerTrafficCounter.incrementIncomingCounter(delta); - } - - /** - * Updates the system counter of written bytes. This information is used by the outgoing - * bytes statistic. - * - * @param session the session that wrote more bytes to the socket. - */ - private void updateWrittenBytesCounter(IoSession session) { - long currentBytes = session.getWrittenBytes(); - Long prevBytes = (Long) session.getAttribute("_written_bytes"); - long delta; - if (prevBytes == null) { - delta = currentBytes; - } - else { - delta = currentBytes - prevBytes; - } - session.setAttribute("_written_bytes", currentBytes); - ServerTrafficCounter.incrementOutgoingCounter(delta); - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java deleted file mode 100644 index 1e558afaa8..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/MultiplexerConnectionHandler.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.session.IoSession; -import org.jivesoftware.openfire.PacketDeliverer; -import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.util.JiveGlobals; -import org.jivesoftware.openfire.XMPPServer; -import org.jivesoftware.openfire.multiplex.MultiplexerPacketDeliverer; -import org.jivesoftware.openfire.net.MultiplexerStanzaHandler; -import org.jivesoftware.openfire.net.StanzaHandler; -import org.jivesoftware.util.SystemProperty; - -/** - * ConnectionHandler that knows which subclass of {@link org.jivesoftware.openfire.net.StanzaHandler} should - * be created and how to build and configure a {@link org.jivesoftware.openfire.nio.NIOConnection}. - * - * @author Gaston Dombiak - * @deprecated Use {@link NettyMultiplexerConnectionHandler} instead - */ -@Deprecated -public class MultiplexerConnectionHandler extends ConnectionHandler { - - /** - * Enable / disable backup delivery of stanzas to other connections in the same connection manager when a stanza - * failed to be delivered on a multiplexer (connection manager) connection. When disabled, stanzas that can not - * be delivered on the connection are discarded. - */ - public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.multiplex.backup-packet-delivery.enabled") - .setDefaultValue(true) - .setDynamic(true) - .build(); - - public MultiplexerConnectionHandler(ConnectionConfiguration configuration) { - super(configuration); - } - - @Override - NIOConnection createNIOConnection(IoSession session) { - final PacketDeliverer backupDeliverer = BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new MultiplexerPacketDeliverer() : null; - return new NIOConnection(session, backupDeliverer, configuration); - } - - @Override - StanzaHandler createStanzaHandler(NIOConnection connection) { - return new MultiplexerStanzaHandler(XMPPServer.getInstance().getPacketRouter(), connection); - } - - @Override - int getMaxIdleTime() { - return JiveGlobals.getIntProperty("xmpp.multiplex.idle", 5 * 60 * 1000) / 1000; - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java deleted file mode 100644 index 9836610943..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NIOConnection.java +++ /dev/null @@ -1,492 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software, 2022-2023 Ignite Realtime Foundation. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.buffer.IoBuffer; -import org.apache.mina.core.filterchain.IoFilterChain; -import org.apache.mina.core.session.IoSession; -import org.apache.mina.filter.compression.CompressionFilter; -import org.apache.mina.filter.ssl.SslFilter; -import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.ConnectionCloseListener; -import org.jivesoftware.openfire.PacketDeliverer; -import org.jivesoftware.openfire.auth.UnauthorizedException; -import org.jivesoftware.openfire.net.StanzaHandler; -import org.jivesoftware.openfire.net.StartTlsFilter; -import org.jivesoftware.openfire.session.LocalSession; -import org.jivesoftware.openfire.session.Session; -import org.jivesoftware.openfire.spi.ConnectionConfiguration; -import org.jivesoftware.openfire.spi.EncryptionArtifactFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.xmpp.packet.Packet; -import org.xmpp.packet.StreamError; - -import javax.annotation.Nullable; -import javax.net.ssl.SSLPeerUnverifiedException; -import javax.net.ssl.SSLSession; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.net.UnknownHostException; -import java.nio.charset.CharsetEncoder; -import java.nio.charset.CodingErrorAction; -import java.nio.charset.StandardCharsets; -import java.security.cert.Certificate; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; - -import static org.jivesoftware.openfire.spi.ConnectionManagerImpl.*; - -/** - * Implementation of {@link Connection} interface specific for NIO connections when using the Apache MINA framework. - * - * @author Gaston Dombiak - * @see Apache MINA - * @deprecated Use {@link NettyConnection} instead - */ -@Deprecated -public class NIOConnection implements Connection { - - private static final Logger Log = LoggerFactory.getLogger(NIOConnection.class); - private ConnectionConfiguration configuration; - - /** - * The utf-8 charset for decoding and encoding XMPP packet streams. - */ - public static final String CHARSET = "UTF-8"; - - private LocalSession session; - private IoSession ioSession; - - final private Map closeListeners = new HashMap<>(); - - /** - * Deliverer to use when the connection is closed or was closed when delivering - * a packet. - */ - private PacketDeliverer backupDeliverer; - private int majorVersion = 1; - private int minorVersion = 0; - private String language = null; - - private boolean usingSelfSignedCertificate; - - private static final ThreadLocal encoder = new ThreadLocalEncoder(); - - /** - * Flag that specifies if the connection should be considered closed. Closing a NIO connection - * is an asynch operation so instead of waiting for the connection to be actually closed just - * keep this flag to avoid using the connection between #close was used and the socket is actually - * closed. - */ - private AtomicReference state = new AtomicReference<>(State.OPEN); - - /** - * Lock used to ensure the integrity of the underlying IoSession (refer to - * https://issues.apache.org/jira/browse/DIRMINA-653 for details) - *

- * This lock can be removed once Openfire guarantees a stable delivery - * order, in which case {@link #deliver(Packet)} won't be called - * concurrently any more, which made this lock necessary in the first place. - *

- */ - private final ReentrantLock ioSessionLock = new ReentrantLock(true); - - public NIOConnection(IoSession session, @Nullable PacketDeliverer packetDeliverer, ConnectionConfiguration configuration ) { - this.ioSession = session; - this.backupDeliverer = packetDeliverer; - this.configuration = configuration; - } - - @Override - public boolean validate() { - if (isClosed()) { - return false; - } - deliverRawText(" "); - return !isClosed(); - } - - @Override - public void registerCloseListener(ConnectionCloseListener listener, Object callback) { - if (isClosed()) { - listener.onConnectionClose(session); - } - else { - closeListeners.put( listener, callback ); - } - } - - @Override - public void removeCloseListener(ConnectionCloseListener listener) { - closeListeners.remove( listener ); - } - - @Override - public byte[] getAddress() throws UnknownHostException { - final SocketAddress remoteAddress = ioSession.getRemoteAddress(); - if (remoteAddress == null) throw new UnknownHostException(); - final InetSocketAddress socketAddress = (InetSocketAddress) remoteAddress; - final InetAddress address = socketAddress.getAddress(); - return address.getAddress(); - } - - @Override - public String getHostAddress() throws UnknownHostException { - final SocketAddress remoteAddress = ioSession.getRemoteAddress(); - if (remoteAddress == null) throw new UnknownHostException(); - final InetSocketAddress socketAddress = (InetSocketAddress) remoteAddress; - final InetAddress inetAddress = socketAddress.getAddress(); - return inetAddress.getHostAddress(); - } - - @Override - public String getHostName() throws UnknownHostException { - final SocketAddress remoteAddress = ioSession.getRemoteAddress(); - if (remoteAddress == null) throw new UnknownHostException(); - final InetSocketAddress socketAddress = (InetSocketAddress) remoteAddress; - final InetAddress inetAddress = socketAddress.getAddress(); - return inetAddress.getHostName(); - } - - @Override - public Certificate[] getLocalCertificates() { - SSLSession sslSession = (SSLSession) ioSession.getAttribute(SslFilter.SSL_SECURED); - if (sslSession != null) { - return sslSession.getLocalCertificates(); - } - return new Certificate[0]; - } - - @Override - public Certificate[] getPeerCertificates() { - try { - SSLSession sslSession = (SSLSession) ioSession.getAttribute(SslFilter.SSL_SECURED); - if (sslSession != null) { - return sslSession.getPeerCertificates(); - } - } catch (SSLPeerUnverifiedException e) { - if (Log.isTraceEnabled()) { - // This is perfectly acceptable when mutual authentication is not enforced by Openfire configuration. - Log.trace( "Peer does not offer certificates in session: " + session, e); - } - } - return new Certificate[0]; - } - - @Override - public Optional getTLSProtocolName() { - return Optional.ofNullable((SSLSession) ioSession.getAttribute(SslFilter.SSL_SECURED)) - .map(SSLSession::getProtocol); - } - - @Override - public Optional getCipherSuiteName() { - return Optional.ofNullable((SSLSession) ioSession.getAttribute(SslFilter.SSL_SECURED)) - .map(SSLSession::getCipherSuite); - } - - @Override - public void setUsingSelfSignedCertificate(boolean isSelfSigned) { - this.usingSelfSignedCertificate = isSelfSigned; - } - - @Override - public boolean isUsingSelfSignedCertificate() { - return usingSelfSignedCertificate; - } - - @Override - @Nullable - public PacketDeliverer getPacketDeliverer() { - return backupDeliverer; - } - - @Override - public void close() { - close(null); - } - - @Override - public void close(@Nullable final StreamError error) { - if (state.compareAndSet(State.OPEN, State.CLOSED)) { - - // Ensure that the state of this connection, its session and the MINA context are eventually closed. - - if (session != null) { - session.setStatus(Session.Status.CLOSED); - } - - String rawEndStream = ""; - if (error != null) { - rawEndStream = error.toXML(); - } - rawEndStream += ""; - - try { - deliverRawText0(rawEndStream); - } catch (Exception e) { - Log.error("Failed to deliver stream close tag: " + e.getMessage()); - } - - try { - ioSession.closeOnFlush(); - } catch (Exception e) { - Log.error("Exception while closing MINA session", e); - } - notifyCloseListeners(); // clean up session, etc. - closeListeners.clear(); - } - } - - @Override - public void systemShutdown() { - close(new StreamError(StreamError.Condition.system_shutdown)); - } - - /** - * Notifies all close listeners that the connection has been closed. - * Used by subclasses to properly finish closing the connection. - */ - private void notifyCloseListeners() { - for( final Map.Entry entry : closeListeners.entrySet() ) - { - if (entry.getKey() != null) { - try { - entry.getKey().onConnectionClose(entry.getValue()); - } catch (Exception e) { - Log.error("Error notifying listener: " + entry.getKey(), e); - } - } - } - } - - @Override - public void init(LocalSession owner) { - session = owner; - } - - @Override - public void reinit(LocalSession owner) { - session = owner; - StanzaHandler stanzaHandler = getStanzaHandler(); - stanzaHandler.setSession(owner); - - // ConnectionCloseListeners are registered with their session instance as a callback object. When re-initializing, - // this object needs to be replaced with the new session instance (or otherwise, the old session will be used - // during the callback. OF-2014 - for ( final Map.Entry entry : closeListeners.entrySet() ) - { - if ( entry.getValue() instanceof LocalSession ) { - entry.setValue( owner ); - } - } - } - - @Override - public boolean isInitialized() { - return session != null && !isClosed(); - } - - protected StanzaHandler getStanzaHandler() { - return (StanzaHandler)ioSession.getAttribute(ConnectionHandler.HANDLER); - } - - @Override - public boolean isClosed() { - return state.get() == State.CLOSED; - } - - @Override - @Deprecated // Remove in Openfire 4.9 or later. - public boolean isSecure() { - return isEncrypted(); - } - - @Override - public boolean isEncrypted() { - return ioSession.getFilterChain().contains(TLS_FILTER_NAME); - } - - @Override - public void deliver(Packet packet) throws UnauthorizedException { - if (isClosed()) { - if (backupDeliverer != null) { - backupDeliverer.deliver(packet); - } else { - Log.trace("Discarding packet that was due to be delivered on closed connection {}, for which no backup deliverer was configured.", this); - } - } - else { - boolean errorDelivering = false; - IoBuffer buffer = IoBuffer.allocate(4096); - buffer.setAutoExpand(true); - try { - buffer.putString(packet.getElement().asXML(), encoder.get()); - buffer.flip(); - - ioSessionLock.lock(); - try { - ioSession.write(buffer); - } finally { - ioSessionLock.unlock(); - } - } - catch (Exception e) { - Log.debug("Error delivering packet:\n" + packet, e); - errorDelivering = true; - } - if (errorDelivering) { - close(); - // Retry sending the packet again. Most probably if the packet is a - // Message it will be stored offline - if (backupDeliverer != null) { - backupDeliverer.deliver(packet); - } else { - Log.trace("Discarding packet that failed to be delivered to connection {}, for which no backup deliverer was configured.", this); - } - } - else { - session.incrementServerPacketCount(); - } - } - } - - @Override - public void deliverRawText(String text) { - if (!isClosed()) { - deliverRawText0(text); - } - } - - private void deliverRawText0(String text){ - boolean errorDelivering = false; - IoBuffer buffer = IoBuffer.allocate(text.length()); - buffer.setAutoExpand(true); - try { - //Charset charset = Charset.forName(CHARSET); - //buffer.putString(text, charset.newEncoder()); - buffer.put(text.getBytes(StandardCharsets.UTF_8)); - buffer.flip(); - ioSessionLock.lock(); - try { - ioSession.write(buffer); - } - finally { - ioSessionLock.unlock(); - } - } - catch (Exception e) { - Log.debug("Error delivering raw text:\n" + text, e); - errorDelivering = true; - } - - // Attempt to close the connection if delivering text fails. - if (errorDelivering) { - close(); - } - } - - public void startTLS(boolean clientMode, boolean directTLS) throws Exception { - - final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); - final SslFilter filter; - if ( clientMode ) - { - filter = factory.createClientModeSslFilter(); - } - else - { - filter = factory.createServerModeSslFilter(); - } - - ioSession.getFilterChain().addBefore(EXECUTOR_FILTER_NAME, TLS_FILTER_NAME, filter); - - if (!directTLS) - { - ioSession.getFilterChain().addAfter(TLS_FILTER_NAME, STARTTLS_FILTER_NAME, new StartTlsFilter()); - } - - if ( !clientMode && !directTLS ) { - // Indicate the client that the server is ready to negotiate TLS - deliverRawText( "" ); - ioSession.getFilterChain().remove(STARTTLS_FILTER_NAME); - } - } - - @Override - public void addCompression() { - IoFilterChain chain = ioSession.getFilterChain(); - String baseFilter = EXECUTOR_FILTER_NAME; - if (chain.contains(TLS_FILTER_NAME)) { - baseFilter = TLS_FILTER_NAME; - } - chain.addAfter(baseFilter, COMPRESSION_FILTER_NAME, new CompressionFilter(true, false, CompressionFilter.COMPRESSION_MAX)); - } - - @Override - public void startCompression() { - CompressionFilter ioFilter = (CompressionFilter) ioSession.getFilterChain().get(COMPRESSION_FILTER_NAME); - ioFilter.setCompressOutbound(true); - } - - @Override - public ConnectionConfiguration getConfiguration() - { - return configuration; - } - - @Override - public int getMajorXMPPVersion() { - return majorVersion; - } - - @Override - public int getMinorXMPPVersion() { - return minorVersion; - } - - @Override - public void setXMPPVersion(int majorVersion, int minorVersion) { - this.majorVersion = majorVersion; - this.minorVersion = minorVersion; - } - - @Override - public boolean isCompressed() { - return ioSession.getFilterChain().contains(COMPRESSION_FILTER_NAME); - } - - @Override - public String toString() { - return super.toString() + " MINA Session: " + ioSession; - } - - private static class ThreadLocalEncoder extends ThreadLocal { - - @Override - protected CharsetEncoder initialValue() { - return StandardCharsets.UTF_8.newEncoder() - .onMalformedInput(CodingErrorAction.REPORT) - .onUnmappableCharacter(CodingErrorAction.REPORT); - } - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java index 15a189dd3f..aa62d7e5b4 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyClientConnectionHandler.java @@ -40,7 +40,7 @@ public class NettyClientConnectionHandler extends NettyConnectionHandler{ * are discarded. */ public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.client.netty-backup-packet-delivery.enabled") + .setKey("xmpp.client.backup-packet-delivery.enabled") .setDefaultValue(true) .setDynamic(true) .build(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java index 41239d117a..e379f36a63 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyComponentConnectionHandler.java @@ -22,7 +22,7 @@ public class NettyComponentConnectionHandler extends NettyConnectionHandler { * component connection. When disabled, stanzas that can not be delivered on the connection are discarded. */ public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.component.netty-backup-packet-delivery.enabled") + .setKey("xmpp.component.backup-packet-delivery.enabled") .setDefaultValue(true) .setDynamic(true) .build(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index 3e2e1aca5e..4138c7fca8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -21,7 +21,6 @@ import io.netty.channel.SimpleChannelInboundHandler; import io.netty.handler.traffic.ChannelTrafficShapingHandler; import io.netty.util.AttributeKey; -import org.apache.mina.core.session.IoSession; import org.dom4j.io.XMPPPacketReader; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.net.MXParser; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java index c8bd0c2f10..e8c6f126a3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyMultiplexerConnectionHandler.java @@ -40,7 +40,7 @@ public class NettyMultiplexerConnectionHandler extends NettyConnectionHandler { * be delivered on the connection are discarded. */ public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.multiplex.netty-backup-packet-delivery.enabled") + .setKey("xmpp.multiplex.backup-packet-delivery.enabled") .setDefaultValue(true) .setDynamic(true) .build(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java index bae6a368d8..4d2e8e13eb 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyServerConnectionHandler.java @@ -58,7 +58,7 @@ public class NettyServerConnectionHandler extends NettyConnectionHandler * server-to-server connection. When disabled, stanzas that can not be delivered on the connection are discarded. */ public static final SystemProperty BACKUP_PACKET_DELIVERY_ENABLED = SystemProperty.Builder.ofType(Boolean.class) - .setKey("xmpp.server.netty-backup-packet-delivery.enabled") + .setKey("xmpp.server.backup-packet-delivery.enabled") .setDefaultValue(true) .setDynamic(true) .build(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/OfflinePacketDeliverer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/OfflinePacketDeliverer.java index 84e2331912..1783688280 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/OfflinePacketDeliverer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/OfflinePacketDeliverer.java @@ -29,7 +29,7 @@ import org.xmpp.packet.Presence; /** - * Fallback method used by {@link org.jivesoftware.openfire.nio.NIOConnection} when a + * Fallback method used by {@link org.jivesoftware.openfire.nio.NettyConnection} when a * connection fails to send a {@link Packet} (likely because it was closed). Message packets * will be stored offline for later retrieval. IQ and Presence packets are dropped.

* diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java deleted file mode 100644 index 7ebd28aef8..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPCodecFactory.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.session.IoSession; -import org.apache.mina.filter.codec.ProtocolCodecFactory; -import org.apache.mina.filter.codec.ProtocolDecoder; -import org.apache.mina.filter.codec.ProtocolEncoder; - -/** - * Factory that specifies the encode and decoder to use for parsing XMPP stanzas. - * - * @author Gaston Dombiak - */ -@Deprecated -public class XMPPCodecFactory implements ProtocolCodecFactory { - - private final XMPPEncoder encoder; - private final XMPPDecoder decoder; - - public XMPPCodecFactory() { - encoder = new XMPPEncoder(); - decoder = new XMPPDecoder(); - } - - @Override - public ProtocolEncoder getEncoder(IoSession session) throws Exception { - return encoder; - } - - @Override - public ProtocolDecoder getDecoder(IoSession session) throws Exception { - return decoder; - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java deleted file mode 100644 index 57b8dff820..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPDecoder.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software, 2023 Ignite Realtime Foundation. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.buffer.IoBuffer; -import org.apache.mina.core.session.IoSession; -import org.apache.mina.filter.codec.CumulativeProtocolDecoder; -import org.apache.mina.filter.codec.ProtocolDecoderOutput; - -import java.nio.CharBuffer; -import java.nio.charset.CharsetDecoder; -import java.nio.charset.CodingErrorAction; -import java.nio.charset.StandardCharsets; - -/** - * Decoder class that parses ByteBuffers and generates XML stanzas. Generated - * stanzas are then passed to the next filters. - * - * @author Gaston Dombiak - * @deprecated Use {@link NettyXMPPDecoder} instead - */ -@Deprecated -public class XMPPDecoder extends CumulativeProtocolDecoder { - - @Override - protected boolean doDecode(IoSession session, IoBuffer in, ProtocolDecoderOutput out) throws Exception { - // Get the XML light parser from the IoSession - XMLLightweightParser parser = (XMLLightweightParser) session.getAttribute(ConnectionHandler.XML_PARSER); - - if (parser.isMaxBufferSizeExceeded()) { - // exception was thrown before, avoid duplicate exception(s) - // "read" and discard remaining data - in.position(in.limit()); - return true; // we are empty - } - - CharBuffer charBuffer = CharBuffer.allocate(in.capacity()); - - CharsetDecoder encoder = StandardCharsets.UTF_8.newDecoder() - .onMalformedInput(CodingErrorAction.REPLACE) - .onUnmappableCharacter(CodingErrorAction.REPLACE); - encoder.reset(); - encoder.decode(in.buf(), charBuffer, false); - - char[] buf = new char[charBuffer.position()]; - charBuffer.flip(); - charBuffer.get(buf); - - // Parse as many stanzas as possible from the received data - parser.read(buf); - - if (parser.areThereMsgs()) { - for (String stanza : parser.getMsgs()) { - out.write(stanza); - } - } - return !in.hasRemaining(); // AKA return isEmpty() - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java deleted file mode 100644 index 7ad8452823..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/XMPPEncoder.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright (C) 2005-2008 Jive Software. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.jivesoftware.openfire.nio; - -import org.apache.mina.core.session.IoSession; -import org.apache.mina.filter.codec.ProtocolEncoderAdapter; -import org.apache.mina.filter.codec.ProtocolEncoderOutput; - -/** - * Encoder that does nothing. We are already writing ByteBuffers so there is no need - * to encode them.

- * - * This class exists as a counterpart of {@link XMPPDecoder}. Unlike that class this class does nothing. - * - * @author Gaston Dombiak - */ -@Deprecated -public class XMPPEncoder extends ProtocolEncoderAdapter { - - @Override - public void encode(IoSession session, Object message, ProtocolEncoderOutput out) - throws Exception { - // Ignore. Do nothing. Content being sent is already a bytebuffer (of strings) - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java index ad6e1f873d..ee729983e3 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionListener.java @@ -15,7 +15,6 @@ */ package org.jivesoftware.openfire.spi; -import org.apache.mina.transport.socket.nio.NioSocketAcceptor; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.ConnectionManager; import org.jivesoftware.openfire.XMPPServer; @@ -194,7 +193,7 @@ public synchronized void enable( boolean enable ) */ public synchronized void start() { - // TODO Start all connection types here, by supplying more connection acceptors other than a MINA-based one. + // TODO Start all connection types here, by supplying more connection acceptors other than a Netty-based one. switch ( getType() ) { case BOSH_C2S: @@ -349,22 +348,6 @@ public synchronized void reloadConfiguration() Log.info( "Reconfigured." ); } - /** - * Returns the MINA-specific socket acceptor that is managed by the instance. - * - * @return A socket acceptor, or null when this listener is disabled or not based on a MINA implementation. - */ - // TODO see if we can avoid exposing MINA internals. - public NioSocketAcceptor getSocketAcceptor() - { - if ( connectionAcceptor == null || !(connectionAcceptor instanceof MINAConnectionAcceptor) ) - { - return null; - } - - return ((MINAConnectionAcceptor)connectionAcceptor).getSocketAcceptor(); - } - /** * Returns the network address on which connections are accepted when this listener is enabled. * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java index 2eac63cbfe..da737589bc 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/ConnectionManagerImpl.java @@ -16,9 +16,6 @@ package org.jivesoftware.openfire.spi; -import org.apache.mina.core.buffer.IoBuffer; -import org.apache.mina.core.buffer.SimpleBufferAllocator; -import org.apache.mina.transport.socket.nio.NioSocketAcceptor; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.ConnectionManager; import org.jivesoftware.openfire.XMPPServer; @@ -604,11 +601,6 @@ public void setPort( ConnectionType type, boolean startInSslMode, int port ) getListener( type, startInSslMode ).setPort( port ); } - // TODO see if we can avoid exposing MINA internals. - public NioSocketAcceptor getSocketAcceptor( ConnectionType type, boolean startInSslMode ) - { - return getListener( type, startInSslMode ).getSocketAcceptor(); - } // ##################################################################### // Certificates events @@ -670,18 +662,6 @@ private void processPropertyValueChange( String property, Map pa // Module management // ##################################################################### - @Override - public void initialize(XMPPServer server) { - super.initialize(server); - - // Check if we need to configure MINA to use Direct or Heap Buffers - // Note: It has been reported that heap buffers are 50% faster than direct buffers - if (JiveGlobals.getBooleanProperty("xmpp.socket.heapBuffer", true)) { - IoBuffer.setUseDirectBuffer(false); - IoBuffer.setAllocator(new SimpleBufferAllocator()); - } - } - @Override public void start() { super.start(); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index dc585cb2a7..c26e332e5d 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -19,7 +19,6 @@ import io.netty.handler.ssl.ClientAuth; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; -import org.apache.mina.filter.ssl.SslFilter; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.jivesoftware.openfire.keystore.OpenfireX509TrustManager; import org.jivesoftware.util.SystemProperty; @@ -398,80 +397,6 @@ public SslContext createClientModeSslContext() throws SSLException, Unrecoverabl .build(); } - /** - * Creates an Apache MINA SslFilter that is configured to use server mode when handshaking. - * - * For Openfire, an engine is of this mode used for most purposes (as Openfire is a server by nature). - * - * Instead of an SSLContext or SSLEngine, Apache MINA uses an SslFilter instance. It is generally not needed to - * create both SSLContext/SSLEngine as well as SslFilter instances. - * - * @return An initialized SslFilter instance (never null) - * @throws KeyManagementException if there was problem manging the ket - * @throws NoSuchAlgorithmException if the algorithm is not supported - * @throws KeyStoreException if there was a problem accessing the keystore - * @throws UnrecoverableKeyException if the key could not be recovered - */ - public SslFilter createServerModeSslFilter() throws KeyManagementException, NoSuchAlgorithmException, KeyStoreException, UnrecoverableKeyException - { - final SSLContext sslContext = getSSLContext(); - final SSLEngine sslEngine = createServerModeSSLEngine(); - - return createSslFilter( sslContext, sslEngine ); - } - - /** - * Creates an Apache MINA SslFilter that is configured to use client mode when handshaking. - * - * For Openfire, a filter of this mode is typically used when the server tries to connect to another server. - * - * Instead of an SSLContext or SSLEngine, Apache MINA uses an SslFilter instance. It is generally not needed to - * create both SSLContext/SSLEngine as well as SslFilter instances. - * - * @return An initialized SslFilter instance (never null) - * @throws KeyManagementException if there was problem manging the ket - * @throws NoSuchAlgorithmException if the algorithm is not supported - * @throws KeyStoreException if there was a problem accessing the keystore - * @throws UnrecoverableKeyException if the key could not be recovered - */ - public SslFilter createClientModeSslFilter() throws KeyManagementException, NoSuchAlgorithmException, KeyStoreException, UnrecoverableKeyException - { - final SSLContext sslContext = getSSLContext(); - final SSLEngine sslEngine = createClientModeSSLEngine(); - - return createSslFilter( sslContext, sslEngine ); - } - - /** - * A utility method that implements the shared functionality of getServerModeSslFilter and getClientModeSslFilter. - * - * This method is used to initialize and configure an instance of SslFilter for a particular pre-configured - * SSLContext and SSLEngine. In most cases, developers will want to use getServerModeSslFilter or - * getClientModeSslFilter instead of this method. - * - * @param sslContext a pre-configured SSL Context instance (cannot be null). - * @param sslEngine a pre-configured SSL Engine instance (cannot be null). - * @return A SslFilter instance (never null). - */ - private static SslFilter createSslFilter( SSLContext sslContext, SSLEngine sslEngine ) { - final SslFilter filter = new SslFilter( sslContext ); - - // Copy configuration from the SSL Engine into the filter. - filter.setEnabledProtocols( sslEngine.getEnabledProtocols() ); - filter.setEnabledCipherSuites( sslEngine.getEnabledCipherSuites() ); - - // Note that the setters for 'need' and 'want' influence each-other. Invoke only one of them! - if ( sslEngine.getNeedClientAuth() ) - { - filter.setNeedClientAuth( true ); - } - else if ( sslEngine.getWantClientAuth() ) - { - filter.setWantClientAuth( true ); - } - return filter; - } - /** * Returns the names of all encryption protocols that are supported (but not necessarily enabled). * diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/LegacyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/LegacyConnectionAcceptor.java deleted file mode 100644 index 87c79f4a26..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/LegacyConnectionAcceptor.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Copyright 2019-2022 Ignite Realtime Foundation. All rights reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.jivesoftware.openfire.spi; - -import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.net.SocketAcceptThread; -import org.jivesoftware.util.SystemProperty; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.net.BindException; -import java.time.Duration; -import java.time.temporal.ChronoUnit; - -/** - * A connection acceptor that employs the legacy, pre-MINA/NIO socket implementation of Openfire. - * - * @author Guus der Kinderen, guus.der.kinderen@gmail.com - * @deprecated Used only for S2S, which should be ported to NIO. - */ -@Deprecated -public class LegacyConnectionAcceptor extends ConnectionAcceptor -{ - private final Logger Log = LoggerFactory.getLogger( LegacyConnectionAcceptor.class ); - - public static final SystemProperty STARTUP_RETRY_DELAY = SystemProperty.Builder.ofType(Duration.class) - .setKey("xmpp.server.startup.retry.delay") - .setDefaultValue(Duration.ofMillis(2000)) - .setMinValue(Duration.ofMillis(-1)) - .setChronoUnit(ChronoUnit.MILLIS) - .setDynamic(Boolean.TRUE) - .build(); - - private SocketAcceptThread socketAcceptThread; - - /** - * Constructs a new instance which will accept new connections based on the provided configuration. - *

- * The provided configuration is expected to be immutable. ConnectionAcceptor instances are not expected to handle - * changes in configuration. When such changes are to be applied, an instance is expected to be replaced. - *

- * Newly instantiated ConnectionAcceptors will not accept any connections before {@link #start()} is invoked. - * - * @param configuration The configuration for connections to be accepted (cannot be null). - */ - public LegacyConnectionAcceptor( ConnectionConfiguration configuration ) - { - super( configuration ); - } - - /** - * Starts this acceptor by binding the socket acceptor. When the acceptor is already started, a warning will be - * logged and the method invocation is otherwise ignored. - */ - @Override - public synchronized void start() - { - if ( socketAcceptThread != null ) - { - Log.warn( "Unable to start acceptor (it is already started!)" ); - return; - } - - if ( configuration.getMaxThreadPoolSize() > 1 ) { - Log.warn( "Configuration allows for up to " + configuration.getMaxThreadPoolSize() + " threads, although implementation is limited to exactly one." ); - } - - doStart(true); - } - - private synchronized void doStart(boolean allowRetry) { - try { - socketAcceptThread = new SocketAcceptThread(configuration.getPort(), configuration.getBindAddress(), configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode); - socketAcceptThread.setDaemon(true); - socketAcceptThread.setPriority(Thread.MAX_PRIORITY); - socketAcceptThread.start(); - } - catch (Exception e) { - System.err.println("Error starting " + configuration.getPort() + ": " + e.getMessage()); - Log.error("Error starting: " + configuration.getPort(), e); - - // Reset for future use. - if (socketAcceptThread != null) { - try { - socketAcceptThread.shutdown(); - } finally { - socketAcceptThread = null; - } - } - - if (allowRetry && e instanceof BindException) { - // OF-1956: Retry a failed bind after a short delay, to work around restart race conditions. - final Duration delay = STARTUP_RETRY_DELAY.getValue(); - if (delay != null && !delay.isNegative()) { - System.err.println("Retrying to start " + configuration.getPort() + " after a delay of " + delay); - Log.error("Retrying to start " + configuration.getPort() + " after a delay of " + delay); - try { - Thread.sleep( delay.toMillis()); - doStart(false); - } catch (InterruptedException ex) { - Log.error("Restart interrupted.", ex); - } - } - } - } - } - - /** - * Stops this acceptor by unbinding the socket acceptor. Does nothing when the instance is not started. - */ - @Override - public synchronized void stop() - { - if ( socketAcceptThread != null ) { - try { - socketAcceptThread.shutdown(); - } finally { - socketAcceptThread = null; - } - } - } - - @Override - public synchronized boolean isIdle() - { - return socketAcceptThread != null; // We're not tracking actual sessions. This is a best effort response. - } - - @Override - public synchronized void reconfigure( ConnectionConfiguration configuration ) - { - this.configuration = configuration; - - // nothing can be reloaded in this implementation. - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java deleted file mode 100644 index 8523e7303f..0000000000 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/MINAConnectionAcceptor.java +++ /dev/null @@ -1,288 +0,0 @@ -package org.jivesoftware.openfire.spi; - -import org.apache.mina.core.filterchain.DefaultIoFilterChainBuilder; -import org.apache.mina.filter.codec.ProtocolCodecFilter; -import org.apache.mina.filter.executor.ExecutorFilter; -import org.apache.mina.filter.ssl.SslFilter; -import org.apache.mina.transport.socket.SocketSessionConfig; -import org.apache.mina.transport.socket.nio.NioSocketAcceptor; -import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.JMXManager; -import org.jivesoftware.openfire.mbean.ThreadPoolExecutorDelegate; -import org.jivesoftware.openfire.mbean.ThreadPoolExecutorDelegateMBean; -import org.jivesoftware.openfire.net.StalledSessionsFilter; -import org.jivesoftware.openfire.nio.*; -import org.jivesoftware.util.JiveGlobals; -import org.jivesoftware.util.NamedThreadFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.management.ObjectName; -import java.net.InetSocketAddress; -import java.security.KeyManagementException; -import java.security.KeyStoreException; -import java.security.NoSuchAlgorithmException; -import java.security.UnrecoverableKeyException; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -/** - * This class is responsible for accepting new (socket) connections, using Java NIO implementation provided by the - * Apache MINA framework. - * - * @author Guus der Kinderen, guus.der.kinderen@gmail.com - */ -class MINAConnectionAcceptor extends ConnectionAcceptor -{ - private final Logger Log; - private final String name; - private final ConnectionHandler connectionHandler; - - private final EncryptionArtifactFactory encryptionArtifactFactory; - - private NioSocketAcceptor socketAcceptor; - - /** - * Object name used to register delegate MBean (JMX) for the thread pool executor. - */ - private ObjectName executorServiceObjectName; - - /** - * Instantiates, but not starts, a new instance. - * @deprecated Use {@link NettyConnectionAcceptor} instead - */ - @Deprecated - public MINAConnectionAcceptor( ConnectionConfiguration configuration ) - { - super( configuration ); - - this.name = configuration.getType().toString().toLowerCase() + ( configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode ? "_ssl" : "" ); - Log = LoggerFactory.getLogger( MINAConnectionAcceptor.class.getName() + "[" + name + "]" ); - - switch ( configuration.getType() ) - { - case COMPONENT: - connectionHandler = new ComponentConnectionHandler( configuration ); - break; - case CONNECTION_MANAGER: - connectionHandler = new MultiplexerConnectionHandler( configuration ); - break; - default: - throw new IllegalStateException( "This implementation does not support the connection type as defined in the provided configuration: " + configuration.getType() ); - } - - this.encryptionArtifactFactory = new EncryptionArtifactFactory( configuration ); - } - - /** - * Starts this acceptor by binding the socket acceptor. When the acceptor is already started, a warning will be - * logged and the method invocation is otherwise ignored. - */ - @Override - public synchronized void start() - { - if ( socketAcceptor != null ) - { - Log.warn( "Unable to start acceptor (it is already started!)" ); - return; - } - - try - { - // Configure the thread pool that is to be used. - final int initialSize = ( configuration.getMaxThreadPoolSize() / 4 ) + 1; - final ExecutorFilter executorFilter = new ExecutorFilter( initialSize, configuration.getMaxThreadPoolSize(), 60, TimeUnit.SECONDS ); - final ThreadPoolExecutor eventExecutor = (ThreadPoolExecutor) executorFilter.getExecutor(); - final ThreadFactory threadFactory = new NamedThreadFactory( name + "-thread-", eventExecutor.getThreadFactory(), true, null ); - eventExecutor.setThreadFactory( threadFactory ); - - // Construct a new socket acceptor, and configure it. - socketAcceptor = buildSocketAcceptor(); - - if (JMXManager.isEnabled()) { - // Register a 'vanilla' Openfire MBean, that provides consistent exposure of thread pool executors. - final ThreadPoolExecutorDelegateMBean mBean = new ThreadPoolExecutorDelegate(eventExecutor); - this.executorServiceObjectName = JMXManager.tryRegister(mBean, ThreadPoolExecutorDelegateMBean.BASE_OBJECT_NAME + name); - } - - final DefaultIoFilterChainBuilder filterChain = socketAcceptor.getFilterChain(); - filterChain.addFirst( ConnectionManagerImpl.EXECUTOR_FILTER_NAME, executorFilter ); - - // Add the XMPP codec filter - filterChain.addAfter( ConnectionManagerImpl.EXECUTOR_FILTER_NAME, ConnectionManagerImpl.XMPP_CODEC_FILTER_NAME, new ProtocolCodecFilter( new XMPPCodecFactory() ) ); - - // Kill sessions whose outgoing queues keep growing and fail to send traffic - filterChain.addAfter( ConnectionManagerImpl.XMPP_CODEC_FILTER_NAME, ConnectionManagerImpl.CAPACITY_FILTER_NAME, new StalledSessionsFilter() ); - - // Ports can be configured to start connections in SSL (as opposed to upgrade a non-encrypted socket to an encrypted one, typically using StartTLS) - if ( configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode ) - { - final SslFilter sslFilter = encryptionArtifactFactory.createServerModeSslFilter(); - filterChain.addAfter( ConnectionManagerImpl.EXECUTOR_FILTER_NAME, ConnectionManagerImpl.TLS_FILTER_NAME, sslFilter ); - } - - // Throttle sessions who send data too fast - if ( configuration.getMaxBufferSize() > 0 ) - { - socketAcceptor.getSessionConfig().setMaxReadBufferSize( configuration.getMaxBufferSize() ); - Log.debug( "Throttling read buffer for connections to max={} bytes", configuration.getMaxBufferSize() ); - } - - // Start accepting connections - socketAcceptor.setHandler( connectionHandler ); - socketAcceptor.bind( new InetSocketAddress( configuration.getBindAddress(), configuration.getPort() ) ); - } - catch ( Exception e ) - { - System.err.println( "Error starting " + configuration.getPort() + ": " + e.getMessage() ); - Log.error( "Error starting: " + configuration.getPort(), e ); - // Reset for future use. - if (executorServiceObjectName != null) { - JMXManager.tryUnregister(executorServiceObjectName); - executorServiceObjectName = null; - } - if (socketAcceptor != null) { - try { - socketAcceptor.unbind(); - } finally { - socketAcceptor = null; - } - } - } - } - - /** - * Stops this acceptor by unbinding the socket acceptor. Does nothing when the instance is not started. - */ - @Override - public synchronized void stop() - { - if (executorServiceObjectName != null) { - JMXManager.tryUnregister(executorServiceObjectName); - executorServiceObjectName = null; - } - if ( socketAcceptor != null ) - { - socketAcceptor.unbind(); - socketAcceptor = null; - } - } - - /** - * Determines if this instance is currently in a state where it is actively serving connections. - * - * @return false when this instance is started and is currently being used to serve connections (otherwise true) - */ - @Override - public synchronized boolean isIdle() - { - return this.socketAcceptor != null && this.socketAcceptor.getManagedSessionCount() == 0; - } - - @Override - public synchronized void reconfigure( ConnectionConfiguration configuration ) - { - this.configuration = configuration; - - if ( socketAcceptor == null ) - { - return; // reconfig will occur when acceptor is started. - } - - final DefaultIoFilterChainBuilder filterChain = socketAcceptor.getFilterChain(); - - if ( filterChain.contains( ConnectionManagerImpl.EXECUTOR_FILTER_NAME ) ) - { - final ExecutorFilter executorFilter = (ExecutorFilter) filterChain.get( ConnectionManagerImpl.EXECUTOR_FILTER_NAME ); - ( (ThreadPoolExecutor) executorFilter.getExecutor()).setCorePoolSize( ( configuration.getMaxThreadPoolSize() / 4 ) + 1 ); - ( (ThreadPoolExecutor) executorFilter.getExecutor()).setMaximumPoolSize( ( configuration.getMaxThreadPoolSize() ) ); - } - - if ( configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode ) - { - // add or replace TLS filter (that's used only for 'direct-TLS') - try - { - final SslFilter sslFilter = encryptionArtifactFactory.createServerModeSslFilter(); - if ( filterChain.contains( ConnectionManagerImpl.TLS_FILTER_NAME ) ) - { - filterChain.replace( ConnectionManagerImpl.TLS_FILTER_NAME, sslFilter ); - } - else - { - filterChain.addAfter( ConnectionManagerImpl.EXECUTOR_FILTER_NAME, ConnectionManagerImpl.TLS_FILTER_NAME, sslFilter ); - } - } - catch ( KeyManagementException | NoSuchAlgorithmException | UnrecoverableKeyException | KeyStoreException e ) - { - Log.error( "An exception occurred while reloading the TLS configuration.", e ); - } - } - else - { - // The acceptor is in 'startTLS' mode. Remove TLS filter (that's used only for 'direct-TLS') - if ( filterChain.contains( ConnectionManagerImpl.TLS_FILTER_NAME ) ) - { - filterChain.remove( ConnectionManagerImpl.TLS_FILTER_NAME ); - } - } - - if ( configuration.getMaxBufferSize() > 0 ) - { - socketAcceptor.getSessionConfig().setMaxReadBufferSize( configuration.getMaxBufferSize() ); - Log.debug( "Throttling read buffer for connections to max={} bytes", configuration.getMaxBufferSize() ); - } - } - - public synchronized int getPort() - { - return configuration.getPort(); - } - - // TODO see if we can avoid exposing MINA internals. - public synchronized NioSocketAcceptor getSocketAcceptor() - { - return socketAcceptor; - } - - private static NioSocketAcceptor buildSocketAcceptor() - { - // Create SocketAcceptor with correct number of processors - final int processorCount = JiveGlobals.getIntProperty( "xmpp.processor.count", Runtime.getRuntime().availableProcessors() ); - - final NioSocketAcceptor socketAcceptor = new NioSocketAcceptor( processorCount ); - - // Set that it will be possible to bind a socket if there is a connection in the timeout state. - socketAcceptor.setReuseAddress( true ); - - // Set the listen backlog (queue) length. Default is 50. - socketAcceptor.setBacklog( JiveGlobals.getIntProperty( "xmpp.socket.backlog", 50 ) ); - - // Set default (low level) settings for new socket connections - final SocketSessionConfig socketSessionConfig = socketAcceptor.getSessionConfig(); - - //socketSessionConfig.setKeepAlive(); - final int receiveBuffer = JiveGlobals.getIntProperty( "xmpp.socket.buffer.receive", -1 ); - if ( receiveBuffer > 0 ) - { - socketSessionConfig.setReceiveBufferSize( receiveBuffer ); - } - - final int sendBuffer = JiveGlobals.getIntProperty( "xmpp.socket.buffer.send", -1 ); - if ( sendBuffer > 0 ) - { - socketSessionConfig.setSendBufferSize( sendBuffer ); - } - - final int linger = JiveGlobals.getIntProperty( "xmpp.socket.linger", -1 ); - if ( linger > 0 ) - { - socketSessionConfig.setSoLinger( linger ); - } - - socketSessionConfig.setTcpNoDelay( JiveGlobals.getBooleanProperty( "xmpp.socket.tcp-nodelay", socketSessionConfig.isTcpNoDelay() ) ); - - return socketAcceptor; - } -} diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/websocket/WebSocketClientConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/websocket/WebSocketClientConnectionHandler.java index 1540c91f08..e950fb9c87 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/websocket/WebSocketClientConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/websocket/WebSocketClientConnectionHandler.java @@ -24,7 +24,7 @@ import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.handler.IQPingHandler; import org.jivesoftware.openfire.net.StanzaHandler; -import org.jivesoftware.openfire.nio.ClientConnectionHandler; +import org.jivesoftware.openfire.nio.NettyClientConnectionHandler; import org.jivesoftware.openfire.nio.OfflinePacketDeliverer; import org.jivesoftware.openfire.session.ConnectionSettings; import org.jivesoftware.openfire.session.LocalClientSession; @@ -45,7 +45,7 @@ /** * This class handles all WebSocket events for the corresponding connection with a remote peer, analogous to the - * function of {@link ClientConnectionHandler} for TCP connections. + * function of {@link NettyClientConnectionHandler} for TCP connections. * * Specifically the XMPP session is managed concurrently with the WebSocket session, including all * framing and authentication requirements. Packets received from the remote peer are forwarded as @@ -102,7 +102,7 @@ public void onConnect(Session session) { lastReceived = Instant.now(); wsSession = session; - final PacketDeliverer backupDeliverer = ClientConnectionHandler.BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; + final PacketDeliverer backupDeliverer = NettyClientConnectionHandler.BACKUP_PACKET_DELIVERY_ENABLED.getValue() ? new OfflinePacketDeliverer() : null; wsConnection = new WebSocketConnection(this, backupDeliverer, session.getRemoteAddress()); websocketFramePingTask = new WebsocketFramePingTask(); if (KEEP_ALIVE_FRAME_PING_ENABLED_PROPERTY.getValue()) { diff --git a/xmppserver/src/main/webapp/session-details.jsp b/xmppserver/src/main/webapp/session-details.jsp index 0c4d732f13..651046046f 100644 --- a/xmppserver/src/main/webapp/session-details.jsp +++ b/xmppserver/src/main/webapp/session-details.jsp @@ -30,7 +30,7 @@ java.util.Collection" errorPage="error.jsp" %> -<%@ page import="org.jivesoftware.openfire.nio.NIOConnection" %> +<%@ page import="org.jivesoftware.openfire.nio.NettyConnection" %> <%@ page import="org.jivesoftware.openfire.websocket.WebSocketConnection" %> <%@ page import="org.jivesoftware.openfire.http.HttpSession" %> <%@ page import="java.util.Map" %> @@ -222,7 +222,7 @@ <% if (s.isDetached()) { %><% - } else if (s.getConnection() instanceof NIOConnection) { + } else if (s.getConnection() instanceof NettyConnection) { %>TCP<% } else if (s.getConnection() instanceof WebSocketConnection) { %>WebSocket<% diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java b/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java index 0d588da9d9..13739ca498 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/nio/XMLLightweightParserTest.java @@ -15,7 +15,8 @@ */ package org.jivesoftware.openfire.nio; -import org.apache.mina.core.buffer.IoBuffer; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -47,9 +48,9 @@ public void setUp() throws Exception { .onUnmappableCharacter(CodingErrorAction.REPLACE); } - private char[] ioBufferToChars(IoBuffer buffer) { + private char[] BytBufToChars(ByteBuf buffer) { CharBuffer charBuffer = CharBuffer.allocate(buffer.capacity()); - encoder.decode(buffer.buf(), charBuffer, false); + encoder.decode(buffer.nioBuffer(), charBuffer, false); char[] buf = new char[charBuffer.position()]; charBuffer.flip(); charBuffer.get(buf); @@ -67,13 +68,13 @@ private char[] ioBufferToChars(IoBuffer buffer) { public void testOF2329OpenAndCloseWithSpace() throws Exception { // Setup test fixture. + final String input = ""; - final IoBuffer buffer = IoBuffer.allocate(input.length(), false); - buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); - buffer.flip(); + ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(input.length()); + buffer.writeBytes(input.getBytes()); // Execute system under test. - parser.read(ioBufferToChars(buffer)); + parser.read(BytBufToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -93,13 +94,12 @@ public void testOF2329OpenAndCloseWithSpaceAndNewline() throws Exception { // Setup test fixture. final String input = ""; - final IoBuffer buffer = IoBuffer.allocate(input.length(), false); - buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); - buffer.flip(); + ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(input.length()); + buffer.writeBytes(input.getBytes()); final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(ioBufferToChars(buffer)); + parser.read(BytBufToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -119,13 +119,12 @@ public void testOF2329OpenAndCloseWithNewline() throws Exception { // Setup test fixture. final String input = ""; - final IoBuffer buffer = IoBuffer.allocate(input.length(), false); - buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); - buffer.flip(); + ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(input.length()); + buffer.writeBytes(input.getBytes()); final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(ioBufferToChars(buffer)); + parser.read(BytBufToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -145,13 +144,12 @@ public void testOF2329SelfTerminatingWithSpace() throws Exception { // Setup test fixture. final String input = ""; - final IoBuffer buffer = IoBuffer.allocate(input.length(), false); - buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); - buffer.flip(); + ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(input.length()); + buffer.writeBytes(input.getBytes()); final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(ioBufferToChars(buffer)); + parser.read(BytBufToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -171,13 +169,12 @@ public void testOF2329SelfTerminatingWithSpaceAndNewline() throws Exception { // Setup test fixture. final String input = ""; - final IoBuffer buffer = IoBuffer.allocate(input.length(), false); - buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); - buffer.flip(); + ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(input.length()); + buffer.writeBytes(input.getBytes()); final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(ioBufferToChars(buffer)); + parser.read(BytBufToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. @@ -197,13 +194,12 @@ public void testOF2329SelfTerminatingWithNewline() throws Exception { // Setup test fixture. final String input = ""; - final IoBuffer buffer = IoBuffer.allocate(input.length(), false); - buffer.putString(input, StandardCharsets.UTF_8.newEncoder()); - buffer.flip(); + ByteBuf buffer = ByteBufAllocator.DEFAULT.buffer(input.length()); + buffer.writeBytes(input.getBytes()); final XMLLightweightParser parser = new XMLLightweightParser(); // Execute system under test. - parser.read(ioBufferToChars(buffer)); + parser.read(BytBufToChars(buffer)); final String[] result = parser.getMsgs(); // Verify results. From 2b95a74bbb4beb2112b86386fe4fea600c23f445 Mon Sep 17 00:00:00 2001 From: Guus der Kinderen Date: Thu, 27 Jul 2023 19:05:27 +0200 Subject: [PATCH 46/56] OF-2632: Do not offer StartTLS when the identity store is empty When the identity store does not contain any certificates, inbound TLS will never be able to succeed. In such cases, lets not advertise the StartTLS feature. --- .../openfire/session/LocalClientSession.java | 15 ++++++++++----- .../LocalConnectionMultiplexerSession.java | 2 +- .../session/LocalIncomingServerSession.java | 5 ++++- 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java index 9ad64c35d9..915f10e1bd 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalClientSession.java @@ -45,6 +45,7 @@ import org.xmpp.packet.StreamError; import java.net.UnknownHostException; +import java.security.KeyStoreException; import java.util.*; /** @@ -274,12 +275,16 @@ public static LocalClientSession createSession(String serverName, XmlPullParser sb = new StringBuilder(490); sb.append(""); - if (connection.getConfiguration().getTlsPolicy() != Connection.TLSPolicy.disabled) { - sb.append(""); - if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required) { - sb.append(""); + try { + if (connection.getConfiguration().getTlsPolicy() != Connection.TLSPolicy.disabled && !connection.getConfiguration().getIdentityStore().getAllCertificates().isEmpty()) { + sb.append(""); + if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required) { + sb.append(""); + } + sb.append(""); } - sb.append(""); + } catch (KeyStoreException e) { + Log.warn("Unable to access the identity store for client connections. StartTLS is not being offered as a feature for this session.", e); } // Include available SASL Mechanisms sb.append(SASLAuthentication.getSASLMechanisms(session)); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java index 24037b4bd6..6b2914a364 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalConnectionMultiplexerSession.java @@ -141,7 +141,7 @@ public static LocalConnectionMultiplexerSession createSession(String serverName, sb = new StringBuilder(490); sb.append(""); - if (connection.getConfiguration().getTlsPolicy() != Connection.TLSPolicy.disabled) { + if (connection.getConfiguration().getTlsPolicy() != Connection.TLSPolicy.disabled && !connection.getConfiguration().getIdentityStore().getAllCertificates().isEmpty()) { sb.append(""); if (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required) { sb.append(""); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java index 056d98be24..7cf32f35dc 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalIncomingServerSession.java @@ -181,7 +181,10 @@ public static LocalIncomingServerSession createSession(String serverName, XmlPul sb.append(""); - if (!directTLS && (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required || connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.optional)) { + if (!directTLS + && (connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required || connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.optional) + && !connection.getConfiguration().getIdentityStore().getAllCertificates().isEmpty() + ) { sb.append(""); if (!ServerDialback.isEnabled()) { Log.debug("Server dialback is disabled so TLS is required"); From b3832d963476afa2eb5ec69aed37cec77987b70f Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Mon, 31 Jul 2023 09:59:24 +0100 Subject: [PATCH 47/56] OF-2559 WIP Fixed TLS negotiation By waiting for handshake to complete before attempting SASL --- .../org/jivesoftware/openfire/Connection.java | 9 ++- .../net/RespondingServerStanzaHandler.java | 73 ++++++++----------- .../openfire/net/SocketConnection.java | 5 +- .../openfire/net/VirtualConnection.java | 5 +- .../openfire/nio/NettyConnection.java | 13 +++- .../openfire/nio/NettyConnectionHandler.java | 20 +++++ .../nio/NettyOutboundConnectionHandler.java | 57 +++++++++++++-- .../session/LocalOutgoingServerSession.java | 4 +- .../LocalOutgoingServerSessionTest.java | 6 +- 9 files changed, 134 insertions(+), 58 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java index 2f8e8cb195..2512382eef 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java @@ -16,6 +16,8 @@ package org.jivesoftware.openfire; +import io.netty.channel.Channel; +import io.netty.util.concurrent.Future; import org.dom4j.Namespace; import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.session.LocalSession; @@ -343,15 +345,16 @@ default boolean isEncrypted() { * will be the server during the TLS negotiation. Therefore, the server requesting the TLS * negotiation must pass true in the {@code clientMode} parameter and the server * receiving the TLS request must pass false in the {@code clientMode} parameter.

- * + *

* In the case of client-2-server the XMPP server must pass false in the * {@code clientMode} parameter since it will behave as the server in the TLS negotiation. * * @param clientMode boolean indicating if this entity is a client or a server in the TLS negotiation. - * @param directTLS boolean indicating if the negotiation is directTLS (true) or startTLS (false). + * @param directTLS boolean indicating if the negotiation is directTLS (true) or startTLS (false). + * @return * @throws Exception if an error occurred while encrypting the connection. */ - void startTLS(boolean clientMode, boolean directTLS) throws Exception; + Future startTLS(boolean clientMode, boolean directTLS) throws Exception; /** * Adds the compression filter to the connection but only filter incoming traffic. Do not filter diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index 1baf94f603..34f61fa29a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -24,9 +24,11 @@ import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.PacketRouter; import org.jivesoftware.openfire.server.ServerDialback; -import org.jivesoftware.openfire.session.*; +import org.jivesoftware.openfire.session.DomainPair; +import org.jivesoftware.openfire.session.LocalOutgoingServerSession; +import org.jivesoftware.openfire.session.LocalSession; +import org.jivesoftware.openfire.session.ServerSession; import org.jivesoftware.openfire.spi.BasicStreamIDFactory; -import org.jivesoftware.util.JiveGlobals; import org.jivesoftware.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +36,7 @@ import org.xmlpull.v1.XmlPullParserException; import org.xmpp.packet.JID; +import java.io.StringReader; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -105,8 +108,17 @@ protected boolean isStartOfStream(String xml) { @Override protected void initiateSession(String stanza, XMPPPacketReader reader) throws Exception { + boolean initialStream = isStartOfStream(stanza); + if (!initialStream) { + // Ignore + return; + } + if (!sessionCreated) { - super.initiateSession(stanza, reader); + sessionCreated = true; + MXParser parser = reader.getXPPParser(); + parser.setInput(new StringReader(stanza)); + createSession(parser); } } @@ -121,6 +133,11 @@ boolean processUnknowPacket(Element doc) { // Handle features if ("features".equals(rootTagName)) { + // Prevent falling back to dialback if we are already authenticated + if (session.isAuthenticated()) { + return true; + } + // Encryption ------ if (shouldUseTls() && remoteFeaturesContainsStartTLS(doc)) { LOG.debug("Both us and the remote server support the STARTTLS feature. Encrypt and authenticate the connection with TLS & SASL..."); @@ -130,6 +147,7 @@ boolean processUnknowPacket(Element doc) { return true; } else if (mustUseTls() && !connection.isEncrypted()) { LOG.debug("I MUST use TLS but I have no StartTLS in features."); + abandonSessionInitiation(); return false; } @@ -151,8 +169,7 @@ boolean processUnknowPacket(Element doc) { connection.deliverRawText(sb.toString()); startedSASL = true; return true; - } else if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { - + } else if (dialbackOffered && (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned())) { // Next, try dialback LOG.debug("Trying to authenticate using dialback."); LOG.debug("[Acting as Originating Server: Authenticate domain: " + domainPair.getLocal() + " with a RS in the domain of: " + domainPair.getRemote() + " (id: " + session.getStreamID() + ")]"); @@ -211,7 +228,6 @@ boolean processUnknowPacket(Element doc) { // SASL was successful so initiate a new stream StringBuilder sb = new StringBuilder(); sb.append(""); - connection.deliverRawText(sb.toString()); - return true; } return false; } + private void abandonSessionInitiation() { + this.setAttemptedAllAuthenticationMethods(true); + this.setSession(null); + } + private boolean shouldUseTls() { return connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.optional || connection.getConfiguration().getTlsPolicy() == Connection.TLSPolicy.required; } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java index a62cc3c6ed..22ff2715bd 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java @@ -18,6 +18,8 @@ import com.jcraft.jzlib.JZlib; import com.jcraft.jzlib.ZOutputStream; +import io.netty.channel.Channel; +import io.netty.util.concurrent.Future; import org.jivesoftware.openfire.*; import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.session.IncomingServerSession; @@ -150,7 +152,7 @@ public TLSStreamHandler getTLSStreamHandler() { return tlsStreamHandler; } - public void startTLS(boolean clientMode, boolean directTLS) throws IOException { + public Future startTLS(boolean clientMode, boolean directTLS) throws IOException { if (!isEncrypted) { isEncrypted = true; @@ -175,6 +177,7 @@ public void startTLS(boolean clientMode, boolean directTLS) throws IOException { writer = new BufferedWriter(new OutputStreamWriter(tlsStreamHandler.getOutputStream(), StandardCharsets.UTF_8)); xmlSerializer = new XMLSocketWriter(writer, this); } + return null; } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java index 4137c6388b..53c8292b66 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java @@ -16,6 +16,8 @@ package org.jivesoftware.openfire.net; +import io.netty.channel.Channel; +import io.netty.util.concurrent.Future; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.ConnectionCloseListener; import org.jivesoftware.openfire.PacketDeliverer; @@ -109,8 +111,9 @@ public PacketDeliverer getPacketDeliverer() { return null; } - public void startTLS(boolean clientMode, boolean directTLS) throws Exception { + public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { //Ignore + return null; } public void addCompression() { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 2bfefe9569..27b9a6f559 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -16,6 +16,7 @@ package org.jivesoftware.openfire.nio; +import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; @@ -24,6 +25,7 @@ import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; import io.netty.handler.traffic.ChannelTrafficShapingHandler; +import io.netty.util.concurrent.Future; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.ConnectionCloseListener; import org.jivesoftware.openfire.PacketDeliverer; @@ -94,6 +96,7 @@ public class NettyConnection implements Connection { * closed. */ private final AtomicReference state = new AtomicReference<>(State.OPEN); + private boolean isEncrypted = false; public NettyConnection(ChannelHandlerContext channelHandlerContext, @Nullable PacketDeliverer packetDeliverer, ConnectionConfiguration configuration ) { this.channelHandlerContext = channelHandlerContext; @@ -306,7 +309,11 @@ public boolean isSecure() { @Override public boolean isEncrypted() { - return channelHandlerContext.channel().pipeline().get(SSL_HANDLER_NAME) != null; + return isEncrypted; + } + + public void setEncrypted(boolean encrypted) { + isEncrypted = encrypted; } @Override @@ -394,7 +401,7 @@ private void updateWrittenBytesCounter(ChannelHandlerContext ctx) { } } - public void startTLS(boolean clientMode, boolean directTLS) throws Exception { + public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); @@ -412,6 +419,8 @@ public void startTLS(boolean clientMode, boolean directTLS) throws Exception { // Indicate the client that the server is ready to negotiate TLS deliverRawText( "" ); } + + return sslHandler.handshakeFuture(); } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index 4138c7fca8..946e21adea 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -19,6 +19,7 @@ import io.netty.channel.ChannelHandler.Sharable; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.ssl.SslHandshakeCompletionEvent; import io.netty.handler.traffic.ChannelTrafficShapingHandler; import io.netty.util.AttributeKey; import org.dom4j.io.XMPPPacketReader; @@ -70,6 +71,8 @@ protected XMPPPacketReader initialValue() */ private static XmlPullParserFactory factory = null; + volatile boolean sslInitDone; + static { try { factory = XmlPullParserFactory.newInstance(MXParser.class.getName(), null); @@ -152,6 +155,23 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { ctx.close(); } + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (!sslInitDone && evt instanceof SslHandshakeCompletionEvent) { + SslHandshakeCompletionEvent e = (SslHandshakeCompletionEvent) evt; + + if (e.isSuccess()) { + sslInitDone = true; + + NettyConnection connection = ctx.channel().attr(NettyConnectionHandler.CONNECTION).get(); + connection.setEncrypted(true); + Log.debug("TLS negotiation was successful."); + ctx.fireChannelActive(); + } + } + + super.userEventTriggered(ctx, evt); + } /** * Updates the system counter of read bytes. This information is used by the incoming diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java index 93614bbccf..9389af7793 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -21,6 +21,7 @@ import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.net.RespondingServerStanzaHandler; +import org.jivesoftware.openfire.net.SASLAuthentication; import org.jivesoftware.openfire.net.StanzaHandler; import org.jivesoftware.openfire.server.ServerDialback; import org.jivesoftware.openfire.session.ConnectionSettings; @@ -44,7 +45,6 @@ public class NettyOutboundConnectionHandler extends NettyConnectionHandler { private static final Logger Log = LoggerFactory.getLogger(NettyOutboundConnectionHandler.class); private final DomainPair domainPair; private final int port; - volatile boolean sslInitDone; public NettyOutboundConnectionHandler(ConnectionConfiguration configuration, DomainPair domainPair, int port) { super(configuration); @@ -101,15 +101,45 @@ public void handlerAdded(ChannelHandlerContext ctx) { public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { if (!sslInitDone && evt instanceof SslHandshakeCompletionEvent) { SslHandshakeCompletionEvent e = (SslHandshakeCompletionEvent) evt; + RespondingServerStanzaHandler stanzaHandler = (RespondingServerStanzaHandler) ctx.channel().attr(NettyConnectionHandler.HANDLER).get(); + if (e.isSuccess()) { sslInitDone = true; + + NettyConnection connection = ctx.channel().attr(NettyConnectionHandler.CONNECTION).get(); + connection.setEncrypted(true); + Log.debug("TLS negotiation was successful. Connection encrypted. Proceeding with authentication..."); + + // If TLS cannot be used for authentication, it is permissible to use another authentication mechanism + // such as dialback. RFC 6120 does not explicitly allow this, as it does not take into account any other + // authentication mechanism other than TLS (it does mention dialback in an interoperability note. However, + // RFC 7590 Section 3.4 writes: "In particular for XMPP server-to-server interactions, it can be reasonable + // for XMPP server implementations to accept encrypted but unauthenticated connections when Server Dialback + // keys [XEP-0220] are used." In short: if Dialback is allowed, unauthenticated TLS is better than no TLS. + if (SASLAuthentication.verifyCertificates(connection.getPeerCertificates(), domainPair.getRemote(), true)) { + Log.debug("SASL authentication will be attempted"); + Log.debug("Send the stream header and wait for response..."); + sendNewStreamHeader(connection); + } else if (JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true)) { + Log.warn("Aborting attempt to create outgoing session as certificate verification failed, and strictCertificateValidation is enabled."); + abandonSession(stanzaHandler); + } else if (ServerDialback.isEnabled() || ServerDialback.isEnabledForSelfSigned()) { + Log.debug("Failed to verify certificates for SASL authentication. Will continue with dialback."); + sendNewStreamHeader(connection); + } else { + Log.warn("Unable to authenticate the connection: Failed to verify certificates for SASL authentication and dialback is not available."); + abandonSession(stanzaHandler); + } + ctx.fireChannelActive(); } else { // SSL Handshake has failed, fall back to dialback - RespondingServerStanzaHandler stanzaHandler = (RespondingServerStanzaHandler) ctx.channel().attr(NettyConnectionHandler.HANDLER).get(); + stanzaHandler.setSession(null); + + System.out.println("SSL HANDSHAKE FAILED: " + e.cause()); if (ServerDialback.isEnabled() && connectionConfigDoesNotRequireTls()) { - Log.debug("Unable to create a new session. Going to try connecting using server dialback as a fallback."); + Log.debug("Unable to create a new TLS session. Going to try connecting using server dialback as a fallback."); // Use server dialback (pre XMPP 1.0) over a plain connection final LocalOutgoingServerSession outgoingSession = new ServerDialback(domainPair).createOutgoingSession(port); @@ -119,11 +149,9 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc stanzaHandler.setSession(outgoingSession); } else { Log.warn("Unable to create a new session: Dialback (as a fallback) failed."); - stanzaHandler.setSession(null); } } else { Log.warn("Unable to create a new session: exhausted all options (not trying dialback as a fallback, as server dialback is disabled by configuration."); - stanzaHandler.setSession(null); } stanzaHandler.setAttemptedAllAuthenticationMethods(true); @@ -133,6 +161,25 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc super.userEventTriggered(ctx, evt); } + private static void abandonSession(RespondingServerStanzaHandler stanzaHandler) { + stanzaHandler.setSession(null); + stanzaHandler.setAttemptedAllAuthenticationMethods(true); + } + + private void sendNewStreamHeader(NettyConnection connection) { + StringBuilder openingStream = new StringBuilder(); + openingStream.append(""); + connection.deliverRawText(openingStream.toString()); + } + private boolean connectionConfigDoesNotRequireTls() { return this.configuration.getTlsPolicy() != Connection.TLSPolicy.required; } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java index c02db66801..bd05d649af 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalOutgoingServerSession.java @@ -70,11 +70,12 @@ public class LocalOutgoingServerSession extends LocalServerSession implements Ou private static final Interner remoteAuthMutex = Interners.newWeakInterner(); + public static final String XMPP_SERVER_SESSION_INITIALISE_TIMEOUT_KEY = "xmpp.server.session.initialise-timeout"; /** * Controls the S2S outgoing session initialise timeout time in seconds */ public static final SystemProperty INITIALISE_TIMEOUT_SECONDS = SystemProperty.Builder.ofType(Duration.class) - .setKey("xmpp.server.session.initialise-timeout") + .setKey(XMPP_SERVER_SESSION_INITIALISE_TIMEOUT_KEY) .setDefaultValue(Duration.ofSeconds(5)) .setChronoUnit(ChronoUnit.SECONDS) .setDynamic(true) @@ -263,7 +264,6 @@ static LocalOutgoingServerSession createOutgoingSession(@Nonnull final DomainPai } catch (Exception e) { // This might be RFC6120, section 5.4.2.2 "Failure Case" or even an unrelated problem. Handle 'normally'. log.warn("An exception occurred while creating a session. Closing connection.", e); - } finally { sessionInitialiser.stop(); } diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java index 0f0e5957b6..7b6c485cc7 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java @@ -161,8 +161,10 @@ public void tearDown() throws Exception @Test void single() throws Exception { - ServerSettings localServerSettings = new ServerSettings(Connection.TLSPolicy.required, ServerSettings.CertificateState.MISSING, true, true); - ServerSettings remoteServerSettings = new ServerSettings(Connection.TLSPolicy.required, ServerSettings.CertificateState.MISSING, true, true); +// - Local Server (Openfire, System under test) Settings: [encryption=optio, certificate=MISSI, strictCertValidation=true, dialback=SUPPO] +// - Remote Server (dummy/mock server) Settings: [encryption=optio, certificate=INVAL, strictCertValidation=true, dialback=SUPPO] + ServerSettings localServerSettings = new ServerSettings(Connection.TLSPolicy.optional, ServerSettings.CertificateState.MISSING, true, true); + ServerSettings remoteServerSettings = new ServerSettings(Connection.TLSPolicy.optional, ServerSettings.CertificateState.INVALID, true, true); outgoingTest(localServerSettings, remoteServerSettings); } From 681c496655bcc68fdd9a6c2d0df94f8a2d6c41ce Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Mon, 31 Jul 2023 11:57:00 +0100 Subject: [PATCH 48/56] fix: OF-2559 - re-implement strictCertificateValidation logic so that LocalOutboundServerSessionTest pass. Also add generic typing for Connection.starttls return type, some tidy up of comments and WIP code. --- .../org/jivesoftware/openfire/Connection.java | 6 +-- .../openfire/net/SocketConnection.java | 4 +- .../openfire/net/VirtualConnection.java | 4 +- .../openfire/nio/NettyConnection.java | 27 +++------- .../nio/NettyOutboundConnectionHandler.java | 51 ++++++++++++------- .../openfire/session/LocalSession.java | 4 +- .../LocalOutgoingServerSessionTest.java | 11 +--- 7 files changed, 49 insertions(+), 58 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java index 2512382eef..d53dc272ce 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java @@ -39,7 +39,7 @@ * * @author Iain Shigeoka */ -public interface Connection extends Closeable { +public interface Connection extends Closeable { /** * When a connection is used to transmit an XML data, the root element of that data can define XML namespaces other * than the ones that are default (eg: 'jabber:client', 'jabber:server', etc). For an XML parser to be able to parse @@ -351,10 +351,10 @@ default boolean isEncrypted() { * * @param clientMode boolean indicating if this entity is a client or a server in the TLS negotiation. * @param directTLS boolean indicating if the negotiation is directTLS (true) or startTLS (false). - * @return + * @return a future that resolves once TLS handshake has completed * @throws Exception if an error occurred while encrypting the connection. */ - Future startTLS(boolean clientMode, boolean directTLS) throws Exception; + Future startTLS(boolean clientMode, boolean directTLS) throws Exception; /** * Adds the compression filter to the connection but only filter incoming traffic. Do not filter diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java index 22ff2715bd..810a5b6f4f 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java @@ -59,7 +59,7 @@ * @author Iain Shigeoka * @deprecated Old, pre NIO / MINA code. Should not be used as NIO offers better performance. Currently only in use for s2s. */ -public class SocketConnection implements Connection { +public class SocketConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(SocketConnection.class); @@ -152,7 +152,7 @@ public TLSStreamHandler getTLSStreamHandler() { return tlsStreamHandler; } - public Future startTLS(boolean clientMode, boolean directTLS) throws IOException { + public Future startTLS(boolean clientMode, boolean directTLS) throws IOException { if (!isEncrypted) { isEncrypted = true; diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java index 53c8292b66..062c31748e 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java @@ -44,7 +44,7 @@ * * @author Gaston Dombiak */ -public abstract class VirtualConnection implements Connection { +public abstract class VirtualConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(VirtualConnection.class); @@ -111,7 +111,7 @@ public PacketDeliverer getPacketDeliverer() { return null; } - public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { + public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { //Ignore return null; } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 27b9a6f559..496d46eb81 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -63,7 +63,7 @@ * @author Matthew Vivian * @author Alex Gidman */ -public class NettyConnection implements Connection { +public class NettyConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(NettyConnection.class); public static final String SSL_HANDLER_NAME = "ssl"; @@ -330,8 +330,9 @@ public void deliver(Packet packet) throws UnauthorizedException { try { ChannelFuture f = channelHandlerContext.writeAndFlush(packet.getElement().asXML()); updateWrittenBytesCounter(channelHandlerContext); - - // TODO handle errors? + // TODO - handle errors more specifically + // Currently errors are handled by the default exceptionCaught method (log error, close channel) + // We can add a new listener to the ChannelFuture f for more specific error handling. } catch (Exception e) { Log.debug("Error delivering packet:\n" + packet, e); @@ -357,25 +358,11 @@ public void deliver(Packet packet) throws UnauthorizedException { public void deliverRawText(String text) { Log.trace("Sending: " + text); if (!isClosed()) { - boolean errorDelivering = false; ChannelFuture f = channelHandlerContext.writeAndFlush(text); updateWrittenBytesCounter(channelHandlerContext); - // TODO handle errors? - -// try { - // TODO don't block, handle errors async with custom ChannelFutureListener -// f.addListener(ChannelFutureListener.FIRE_EXCEPTION_ON_FAILURE); // Removed the sync so this won't throw -// } -// catch (Exception e) { -// Log.error("Error delivering raw text:\n" + text, e); -// e.printStackTrace(); -// errorDelivering = true; -// } - - // Attempt to close the connection if delivering text fails. -// if (errorDelivering) { -// close(); -// } + // TODO - handle errors more specifically + // Currently errors are handled by the default exceptionCaught method (log error, close channel) + // We can add a new listener to the ChannelFuture f for more specific error handling. } } diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java index 9389af7793..82a2657471 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -32,7 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.security.cert.CertificateException; +import javax.net.ssl.SSLHandshakeException; /** * Outbound (S2S) specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created @@ -68,24 +68,10 @@ StanzaHandler createStanzaHandler(NettyConnection connection) { return new RespondingServerStanzaHandler( XMPPServer.getInstance().getPacketRouter(), connection, domainPair ); } - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - super.exceptionCaught(ctx, cause); - - if (isCertificateException(cause) && configRequiresStrictCertificateValidation()) { - Log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); - throw new RuntimeException(cause); - } - } - private static boolean configRequiresStrictCertificateValidation() { return JiveGlobals.getBooleanProperty(ConnectionSettings.Server.STRICT_CERTIFICATE_VALIDATION, true); } - public boolean isCertificateException(Throwable cause) { - return cause instanceof CertificateException; - } - @Override public int getMaxIdleTime() { return JiveGlobals.getIntProperty(ConnectionSettings.Server.IDLE_TIMEOUT_PROPERTY, 360); @@ -97,13 +83,24 @@ public void handlerAdded(ChannelHandlerContext ctx) { super.handlerAdded(ctx); } + /** + * Called when SSL Handshake has been completed. + * + * If successful, attempts authentication via SASL, or dialback dependent on configuration and certificate validity. + * If not successful, either attempts dialback on a plain un-encrypted connection, or throws an exception dependent + * on configuration. + * + * @param ctx ChannelHandlerContext for the Netty channel + * @param evt Event that has been triggered - this implementation specifically identifies SslHandshakeCompletionEvent + * @throws Exception + */ @Override public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { if (!sslInitDone && evt instanceof SslHandshakeCompletionEvent) { - SslHandshakeCompletionEvent e = (SslHandshakeCompletionEvent) evt; + SslHandshakeCompletionEvent event = (SslHandshakeCompletionEvent) evt; RespondingServerStanzaHandler stanzaHandler = (RespondingServerStanzaHandler) ctx.channel().attr(NettyConnectionHandler.HANDLER).get(); - if (e.isSuccess()) { + if (event.isSuccess()) { sslInitDone = true; NettyConnection connection = ctx.channel().attr(NettyConnectionHandler.CONNECTION).get(); @@ -133,11 +130,19 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc ctx.fireChannelActive(); } else { - // SSL Handshake has failed, fall back to dialback + // SSL Handshake has failed stanzaHandler.setSession(null); - System.out.println("SSL HANDSHAKE FAILED: " + e.cause()); + if (isSSLHandshakeException(event) && isCausedByCertificateError(event)){ + if (configRequiresStrictCertificateValidation()) { + Log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); + throw new RuntimeException(event.cause()); + } else { + Log.warn("TLS handshake failed due to a certificate validation error"); + } + } + // fall back to dialback if (ServerDialback.isEnabled() && connectionConfigDoesNotRequireTls()) { Log.debug("Unable to create a new TLS session. Going to try connecting using server dialback as a fallback."); @@ -161,6 +166,14 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc super.userEventTriggered(ctx, evt); } + private static boolean isSSLHandshakeException(SslHandshakeCompletionEvent event) { + return event.cause() instanceof SSLHandshakeException; + } + + private static boolean isCausedByCertificateError(SslHandshakeCompletionEvent event) { + return event.cause().getMessage().contains("java.security.cert.CertPathBuilderException"); + } + private static void abandonSession(RespondingServerStanzaHandler stanzaHandler) { stanzaHandler.setSession(null); stanzaHandler.setAttemptedAllAuthenticationMethods(true); diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java index 4a10f157e7..362280ea49 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java @@ -570,7 +570,7 @@ public boolean isUsingSelfSignedCertificate() { public String getTLSProtocolName() { return Optional.ofNullable(conn) .map(c -> c.getTLSProtocolName().orElse("NONE")) - .orElse("NONE"); + .orElse("NONE").toString(); } @Override @@ -578,7 +578,7 @@ public String getTLSProtocolName() { public String getCipherSuiteName() { return Optional.ofNullable(conn) .map(c -> c.getCipherSuiteName().orElse("NONE")) - .orElse("NONE"); + .orElse("NONE").toString(); } @Override diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java index 7b6c485cc7..b02b21854c 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java @@ -157,16 +157,7 @@ public void tearDown() throws Exception Fixtures.clearExistingProperties(); } - - - @Test - void single() throws Exception { -// - Local Server (Openfire, System under test) Settings: [encryption=optio, certificate=MISSI, strictCertValidation=true, dialback=SUPPO] -// - Remote Server (dummy/mock server) Settings: [encryption=optio, certificate=INVAL, strictCertValidation=true, dialback=SUPPO] - ServerSettings localServerSettings = new ServerSettings(Connection.TLSPolicy.optional, ServerSettings.CertificateState.MISSING, true, true); - ServerSettings remoteServerSettings = new ServerSettings(Connection.TLSPolicy.optional, ServerSettings.CertificateState.INVALID, true, true); - outgoingTest(localServerSettings, remoteServerSettings); - } + /** * Unit test in which Openfire initiates an outgoing server-to-server connection. From f591cc9121c706831f15e1f8bdb5b3035b816364 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Mon, 31 Jul 2023 12:01:28 +0100 Subject: [PATCH 49/56] chore: OF-2559 - remove unnecessary System.out statements --- .../openfire/net/RespondingServerStanzaHandler.java | 1 - .../openfire/session/RemoteReceivingServerDummy.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java index 34f61fa29a..ef6f806819 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/RespondingServerStanzaHandler.java @@ -241,7 +241,6 @@ boolean processUnknowPacket(Element doc) { if (session instanceof LocalOutgoingServerSession) { ((LocalOutgoingServerSession) session).setAuthenticationMethod(ServerSession.AuthenticationMethod.SASL_EXTERNAL); } else { - System.out.println("Expected session to be a LocalOutgoingServerSession but it isn't, unable to setAuthenticationMethod()."); LOG.debug("Expected session to be a LocalOutgoingServerSession but it isn't, unable to setAuthenticationMethod()."); return false; } diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java b/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java index 9b541a31e5..75f357e5d2 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/session/RemoteReceivingServerDummy.java @@ -50,7 +50,7 @@ public class RemoteReceivingServerDummy extends AbstractRemoteServerDummy implem /** * When switched to 'true', most XMPP interaction will be printed to standard-out. */ - public static final boolean doLog = true; + public static final boolean doLog = false; private ServerSocket server; From edec92b72045f23eb436af57a465a7888019b663 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Mon, 31 Jul 2023 14:21:11 +0100 Subject: [PATCH 50/56] fix: OF-2599 - switch SSLHandshakeException check to a CertificateException check --- .../openfire/nio/NettyOutboundConnectionHandler.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java index 82a2657471..616100711b 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyOutboundConnectionHandler.java @@ -32,7 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.net.ssl.SSLHandshakeException; +import java.security.cert.CertificateException; /** * Outbound (S2S) specific ConnectionHandler that knows which subclass of {@link StanzaHandler} should be created @@ -133,7 +133,7 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc // SSL Handshake has failed stanzaHandler.setSession(null); - if (isSSLHandshakeException(event) && isCausedByCertificateError(event)){ + if (isCertificateException(event)){ if (configRequiresStrictCertificateValidation()) { Log.warn("Aborting attempt to create outgoing session as TLS handshake failed, and strictCertificateValidation is enabled."); throw new RuntimeException(event.cause()); @@ -166,12 +166,8 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc super.userEventTriggered(ctx, evt); } - private static boolean isSSLHandshakeException(SslHandshakeCompletionEvent event) { - return event.cause() instanceof SSLHandshakeException; - } - - private static boolean isCausedByCertificateError(SslHandshakeCompletionEvent event) { - return event.cause().getMessage().contains("java.security.cert.CertPathBuilderException"); + private static boolean isCertificateException(SslHandshakeCompletionEvent event) { + return event.cause().getCause() instanceof CertificateException; } private static void abandonSession(RespondingServerStanzaHandler stanzaHandler) { From 8d8dd7f10a72c729df9e2b5c19a4c8c3a520e239 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 1 Aug 2023 09:17:35 +0100 Subject: [PATCH 51/56] OF-2559 WIP startTLS does not need to return --- .../main/java/org/jivesoftware/openfire/Connection.java | 7 ++----- .../org/jivesoftware/openfire/net/SocketConnection.java | 7 ++----- .../org/jivesoftware/openfire/net/VirtualConnection.java | 7 ++----- .../org/jivesoftware/openfire/nio/NettyConnection.java | 7 ++----- .../openfire/session/LocalOutgoingServerSessionTest.java | 7 ++++--- 5 files changed, 12 insertions(+), 23 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java index d53dc272ce..c564e7ba54 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java @@ -16,8 +16,6 @@ package org.jivesoftware.openfire; -import io.netty.channel.Channel; -import io.netty.util.concurrent.Future; import org.dom4j.Namespace; import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.session.LocalSession; @@ -39,7 +37,7 @@ * * @author Iain Shigeoka */ -public interface Connection extends Closeable { +public interface Connection extends Closeable { /** * When a connection is used to transmit an XML data, the root element of that data can define XML namespaces other * than the ones that are default (eg: 'jabber:client', 'jabber:server', etc). For an XML parser to be able to parse @@ -351,10 +349,9 @@ default boolean isEncrypted() { * * @param clientMode boolean indicating if this entity is a client or a server in the TLS negotiation. * @param directTLS boolean indicating if the negotiation is directTLS (true) or startTLS (false). - * @return a future that resolves once TLS handshake has completed * @throws Exception if an error occurred while encrypting the connection. */ - Future startTLS(boolean clientMode, boolean directTLS) throws Exception; + void startTLS(boolean clientMode, boolean directTLS) throws Exception; /** * Adds the compression filter to the connection but only filter incoming traffic. Do not filter diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java index 810a5b6f4f..a62cc3c6ed 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java @@ -18,8 +18,6 @@ import com.jcraft.jzlib.JZlib; import com.jcraft.jzlib.ZOutputStream; -import io.netty.channel.Channel; -import io.netty.util.concurrent.Future; import org.jivesoftware.openfire.*; import org.jivesoftware.openfire.auth.UnauthorizedException; import org.jivesoftware.openfire.session.IncomingServerSession; @@ -59,7 +57,7 @@ * @author Iain Shigeoka * @deprecated Old, pre NIO / MINA code. Should not be used as NIO offers better performance. Currently only in use for s2s. */ -public class SocketConnection implements Connection { +public class SocketConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(SocketConnection.class); @@ -152,7 +150,7 @@ public TLSStreamHandler getTLSStreamHandler() { return tlsStreamHandler; } - public Future startTLS(boolean clientMode, boolean directTLS) throws IOException { + public void startTLS(boolean clientMode, boolean directTLS) throws IOException { if (!isEncrypted) { isEncrypted = true; @@ -177,7 +175,6 @@ public Future startTLS(boolean clientMode, boolean directTLS) throws IOExc writer = new BufferedWriter(new OutputStreamWriter(tlsStreamHandler.getOutputStream(), StandardCharsets.UTF_8)); xmlSerializer = new XMLSocketWriter(writer, this); } - return null; } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java index 062c31748e..f68782724f 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java @@ -16,8 +16,6 @@ package org.jivesoftware.openfire.net; -import io.netty.channel.Channel; -import io.netty.util.concurrent.Future; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.ConnectionCloseListener; import org.jivesoftware.openfire.PacketDeliverer; @@ -44,7 +42,7 @@ * * @author Gaston Dombiak */ -public abstract class VirtualConnection implements Connection { +public abstract class VirtualConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(VirtualConnection.class); @@ -111,9 +109,8 @@ public PacketDeliverer getPacketDeliverer() { return null; } - public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { + public void startTLS(boolean clientMode, boolean directTLS) throws Exception { //Ignore - return null; } public void addCompression() { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 496d46eb81..2a63977408 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -25,7 +25,6 @@ import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; import io.netty.handler.traffic.ChannelTrafficShapingHandler; -import io.netty.util.concurrent.Future; import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.ConnectionCloseListener; import org.jivesoftware.openfire.PacketDeliverer; @@ -63,7 +62,7 @@ * @author Matthew Vivian * @author Alex Gidman */ -public class NettyConnection implements Connection { +public class NettyConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(NettyConnection.class); public static final String SSL_HANDLER_NAME = "ssl"; @@ -388,7 +387,7 @@ private void updateWrittenBytesCounter(ChannelHandlerContext ctx) { } } - public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { + public void startTLS(boolean clientMode, boolean directTLS) throws Exception { final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); @@ -406,8 +405,6 @@ public Future startTLS(boolean clientMode, boolean directTLS) throws Ex // Indicate the client that the server is ready to negotiate TLS deliverRawText( "" ); } - - return sslHandler.handshakeFuture(); } @Override diff --git a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java index b02b21854c..2a678e8a86 100644 --- a/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java +++ b/xmppserver/src/test/java/org/jivesoftware/openfire/session/LocalOutgoingServerSessionTest.java @@ -16,16 +16,17 @@ package org.jivesoftware.openfire.session; import org.jivesoftware.Fixtures; -import org.jivesoftware.openfire.*; +import org.jivesoftware.openfire.Connection; +import org.jivesoftware.openfire.ConnectionManager; +import org.jivesoftware.openfire.XMPPServer; import org.jivesoftware.openfire.keystore.*; -import org.jivesoftware.openfire.net.*; +import org.jivesoftware.openfire.net.DNSUtil; import org.jivesoftware.openfire.spi.ConnectionConfiguration; import org.jivesoftware.openfire.spi.ConnectionListener; import org.jivesoftware.openfire.spi.ConnectionType; import org.jivesoftware.util.JiveGlobals; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; From e4dd9df113fa5afeef83e5fc3363107a04d26eb5 Mon Sep 17 00:00:00 2001 From: Alex Gidman Date: Tue, 1 Aug 2023 09:05:19 +0100 Subject: [PATCH 52/56] fix: OF-2599 - remove generic type on connection --- .../src/main/java/org/jivesoftware/openfire/Connection.java | 4 ++-- .../java/org/jivesoftware/openfire/net/SocketConnection.java | 5 ++--- .../org/jivesoftware/openfire/net/VirtualConnection.java | 5 ++--- .../java/org/jivesoftware/openfire/nio/NettyConnection.java | 5 ++--- .../java/org/jivesoftware/openfire/session/LocalSession.java | 4 ++-- 5 files changed, 10 insertions(+), 13 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java index d53dc272ce..76a530e1c6 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/Connection.java @@ -39,7 +39,7 @@ * * @author Iain Shigeoka */ -public interface Connection extends Closeable { +public interface Connection extends Closeable { /** * When a connection is used to transmit an XML data, the root element of that data can define XML namespaces other * than the ones that are default (eg: 'jabber:client', 'jabber:server', etc). For an XML parser to be able to parse @@ -354,7 +354,7 @@ default boolean isEncrypted() { * @return a future that resolves once TLS handshake has completed * @throws Exception if an error occurred while encrypting the connection. */ - Future startTLS(boolean clientMode, boolean directTLS) throws Exception; + void startTLS(boolean clientMode, boolean directTLS) throws Exception; /** * Adds the compression filter to the connection but only filter incoming traffic. Do not filter diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java index 810a5b6f4f..e7fc92d4a8 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/SocketConnection.java @@ -59,7 +59,7 @@ * @author Iain Shigeoka * @deprecated Old, pre NIO / MINA code. Should not be used as NIO offers better performance. Currently only in use for s2s. */ -public class SocketConnection implements Connection { +public class SocketConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(SocketConnection.class); @@ -152,7 +152,7 @@ public TLSStreamHandler getTLSStreamHandler() { return tlsStreamHandler; } - public Future startTLS(boolean clientMode, boolean directTLS) throws IOException { + public void startTLS(boolean clientMode, boolean directTLS) throws IOException { if (!isEncrypted) { isEncrypted = true; @@ -177,7 +177,6 @@ public Future startTLS(boolean clientMode, boolean directTLS) throws IOExc writer = new BufferedWriter(new OutputStreamWriter(tlsStreamHandler.getOutputStream(), StandardCharsets.UTF_8)); xmlSerializer = new XMLSocketWriter(writer, this); } - return null; } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java index 062c31748e..ba2c2d138e 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/net/VirtualConnection.java @@ -44,7 +44,7 @@ * * @author Gaston Dombiak */ -public abstract class VirtualConnection implements Connection { +public abstract class VirtualConnection implements Connection { private static final Logger Log = LoggerFactory.getLogger(VirtualConnection.class); @@ -111,9 +111,8 @@ public PacketDeliverer getPacketDeliverer() { return null; } - public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { + public void startTLS(boolean clientMode, boolean directTLS) throws Exception { //Ignore - return null; } public void addCompression() { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java index 496d46eb81..69403f69b6 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnection.java @@ -63,7 +63,7 @@ * @author Matthew Vivian * @author Alex Gidman */ -public class NettyConnection implements Connection { +public class NettyConnection implements Connection{ private static final Logger Log = LoggerFactory.getLogger(NettyConnection.class); public static final String SSL_HANDLER_NAME = "ssl"; @@ -388,7 +388,7 @@ private void updateWrittenBytesCounter(ChannelHandlerContext ctx) { } } - public Future startTLS(boolean clientMode, boolean directTLS) throws Exception { + public void startTLS(boolean clientMode, boolean directTLS) throws Exception { final EncryptionArtifactFactory factory = new EncryptionArtifactFactory( configuration ); @@ -407,7 +407,6 @@ public Future startTLS(boolean clientMode, boolean directTLS) throws Ex deliverRawText( "" ); } - return sslHandler.handshakeFuture(); } @Override diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java index 362280ea49..4a10f157e7 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/session/LocalSession.java @@ -570,7 +570,7 @@ public boolean isUsingSelfSignedCertificate() { public String getTLSProtocolName() { return Optional.ofNullable(conn) .map(c -> c.getTLSProtocolName().orElse("NONE")) - .orElse("NONE").toString(); + .orElse("NONE"); } @Override @@ -578,7 +578,7 @@ public String getTLSProtocolName() { public String getCipherSuiteName() { return Optional.ofNullable(conn) .map(c -> c.getCipherSuiteName().orElse("NONE")) - .orElse("NONE").toString(); + .orElse("NONE"); } @Override From 381286dd3e423f7bf7e9a998d202b4501dc82b65 Mon Sep 17 00:00:00 2001 From: Guus der Kinderen Date: Tue, 1 Aug 2023 12:53:10 +0200 Subject: [PATCH 53/56] OF-2559: Remove MINA artifacts from pom.xml --- pom.xml | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/pom.xml b/pom.xml index fced1353ca..9f1223cd78 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,6 @@ 9.4.43.v20210629 1.2.5 - 2.2.1 4.1.94.Final 1.70 1.7.36 @@ -346,18 +345,6 @@ true - - - - org.apache.felix - maven-bundle-plugin - 4.2.0 - true - From 7c9f90d2f96628e0ee9cf5c54ff8257a4b5537b1 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 1 Aug 2023 12:06:54 +0100 Subject: [PATCH 54/56] OF-2559 Create new business logic handler for each session Prior to this commit handlers were being shared across all sessions. Now a new handler is instantiated per connection/session. --- .../openfire/nio/NettyConnectionHandler.java | 5 ++-- .../openfire/spi/NettyConnectionAcceptor.java | 6 +---- .../openfire/spi/NettyServerInitializer.java | 23 +++++++++++-------- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java index 946e21adea..f0d632054a 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyConnectionHandler.java @@ -16,7 +16,6 @@ package org.jivesoftware.openfire.nio; -import io.netty.channel.ChannelHandler.Sharable; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; import io.netty.handler.ssl.SslHandshakeCompletionEvent; @@ -35,6 +34,7 @@ import org.xmpp.packet.StreamError; import static org.jivesoftware.openfire.spi.NettyServerInitializer.TRAFFIC_HANDLER_NAME; + /** * A NettyConnectionHandler is responsible for creating new sessions, destroying sessions and delivering * received XML stanzas to the proper StanzaHandler.

@@ -45,7 +45,6 @@ * @author Matthew Vivian * @author Alex Gidman */ -@Sharable public abstract class NettyConnectionHandler extends SimpleChannelInboundHandler { private static final Logger Log = LoggerFactory.getLogger(NettyConnectionHandler.class); @@ -71,7 +70,7 @@ protected XMPPPacketReader initialValue() */ private static XmlPullParserFactory factory = null; - volatile boolean sslInitDone; + protected boolean sslInitDone; static { try { diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java index 25c1935536..8b446d6f91 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyConnectionAcceptor.java @@ -26,8 +26,6 @@ import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.util.concurrent.GlobalEventExecutor; import org.jivesoftware.openfire.Connection; -import org.jivesoftware.openfire.nio.NettyConnectionHandler; -import org.jivesoftware.openfire.nio.NettyConnectionHandlerFactory; import org.jivesoftware.util.JiveGlobals; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,7 +73,6 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { private final ChannelGroup allChannels = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); private final Logger Log; - private final NettyConnectionHandler connectionHandler; private Channel mainChannel; /** @@ -83,7 +80,6 @@ class NettyConnectionAcceptor extends ConnectionAcceptor { */ public NettyConnectionAcceptor(ConnectionConfiguration configuration) { super(configuration); - this.connectionHandler = NettyConnectionHandlerFactory.createConnectionHandler(configuration); String name = configuration.getType().toString().toLowerCase() + (isDirectTLSConfigured() ? "_ssl" : ""); Log = LoggerFactory.getLogger( NettyConnectionAcceptor.class.getName() + "[" + name + "]" ); @@ -104,7 +100,7 @@ public synchronized void start() { // Instantiate a new Channel to accept incoming connections. .channel(NioServerSocketChannel.class) // The handler specified here will always be evaluated by a newly accepted Channel. - .childHandler(new NettyServerInitializer(connectionHandler, isDirectTLSConfigured(), allChannels)) + .childHandler(new NettyServerInitializer(configuration, allChannels)) // Set the listen backlog (queue) length. .option(ChannelOption.SO_BACKLOG, JiveGlobals.getIntProperty("xmpp.socket.backlog", 50)) // option() is for the NioServerSocketChannel that accepts incoming connections. diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java index 0e2eae474b..d9ae076647 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/NettyServerInitializer.java @@ -9,6 +9,7 @@ import io.netty.handler.timeout.WriteTimeoutHandler; import io.netty.handler.traffic.ChannelTrafficShapingHandler; +import org.jivesoftware.openfire.Connection; import org.jivesoftware.openfire.nio.*; import org.jivesoftware.util.SystemProperty; @@ -32,21 +33,21 @@ public class NettyServerInitializer extends ChannelInitializer { .setDynamic(true) .build(); - public static final String TRAFFIC_HANDLER_NAME = "trafficShapingHandler" - ; - private final NettyConnectionHandler businessLogicHandler; - private final boolean directTLS; + public static final String TRAFFIC_HANDLER_NAME = "trafficShapingHandler"; private final ChannelGroup allChannels; + private final ConnectionConfiguration configuration; - public NettyServerInitializer(NettyConnectionHandler businessLogicHandler, boolean directTLS, ChannelGroup allChannels) { - this.businessLogicHandler = businessLogicHandler; - this.directTLS = directTLS; + public NettyServerInitializer(ConnectionConfiguration configuration, ChannelGroup allChannels) { this.allChannels = allChannels; + this.configuration = configuration; } @Override public void initChannel(SocketChannel ch) throws Exception { - boolean isClientConnection = businessLogicHandler instanceof NettyClientConnectionHandler; + + boolean isClientConnection = configuration.getType() == ConnectionType.SOCKET_C2S; + + NettyConnectionHandler businessLogicHandler = NettyConnectionHandlerFactory.createConnectionHandler(configuration); int maxIdleTimeBeforeClosing = businessLogicHandler.getMaxIdleTime() > -1 ? businessLogicHandler.getMaxIdleTime() : 0; int maxIdleTimeBeforePinging = maxIdleTimeBeforeClosing / 2; @@ -59,10 +60,14 @@ public void initChannel(SocketChannel ch) throws Exception { .addLast("keepAliveHandler", new NettyIdleStateKeepAliveHandler(isClientConnection)) .addLast(businessLogicHandler); - if (directTLS) { + if (isDirectTLSConfigured()) { ch.attr(CONNECTION).get().startTLS(false, true); } allChannels.add(ch); } + + private boolean isDirectTLSConfigured() { + return this.configuration.getTlsPolicy() == Connection.TLSPolicy.legacyMode; + } } From 0dbf6299a845af372fe16c0f62bc9ba2d1478173 Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 1 Aug 2023 12:10:51 +0100 Subject: [PATCH 55/56] OF-2559 Pass netty events along pipeline Prior to this commit SSL Handshake events were not making it down the netty pipeline to our client connection handler (aka business logic handler). This meant that inbound connections were never set to encrypted=true causing the session to be abandoned when TLS was required. --- .../openfire/nio/NettyIdleStateKeepAliveHandler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java index 7576497adb..061c93a57d 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/nio/NettyIdleStateKeepAliveHandler.java @@ -86,6 +86,7 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc sendPingPacket(ctx); } } + super.userEventTriggered(ctx, evt); } /** From 5e06b1e66a1d133290005a722cb7522a6e33a96f Mon Sep 17 00:00:00 2001 From: Matthew Vivian Date: Tue, 1 Aug 2023 12:13:05 +0100 Subject: [PATCH 56/56] OF-2559 Prevent old ssl engine from using TLS 1.3 The old implementation (still used by ServerDialback) is unable to negotiate a TLS 1.3 connection. Netty-based connections can use TLS 1.3. --- .../jivesoftware/openfire/spi/EncryptionArtifactFactory.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java index c26e332e5d..59dfe466a6 100644 --- a/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java +++ b/xmppserver/src/main/java/org/jivesoftware/openfire/spi/EncryptionArtifactFactory.java @@ -208,7 +208,8 @@ private SSLEngine createSSLEngine() throws UnrecoverableKeyException, NoSuchAlgo final SSLEngine sslEngine = sslContext.createSSLEngine(); // Configure protocol support. - final Set protocols = configuration.getEncryptionProtocols(); + final Set protocols = new HashSet<>(configuration.getEncryptionProtocols()); + protocols.remove("TLSv1.3"); // TLSv1.3 is not compatible with this SSLEngine implementation if ( !protocols.isEmpty() ) { // When an explicit list of enabled protocols is defined, use only those (otherwise, an implementation-specific default will be used).