ARTEMIS-3445 automatically clean-up abandoned MQTT subscriptions

This commit is contained in:
Justin Bertram 2021-08-19 10:32:27 -05:00 committed by Clebert Suconic
parent 9b074e124e
commit 6ee7e72db1
12 changed files with 180 additions and 15 deletions

View File

@ -21,6 +21,7 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
@ -58,25 +59,31 @@ public class MQTTProtocolManager extends AbstractProtocolManager<MqttMessage, MQ
private final List<MQTTInterceptor> incomingInterceptors = new ArrayList<>();
private final List<MQTTInterceptor> outgoingInterceptors = new ArrayList<>();
//TODO Read in a list of existing client IDs from stored Sessions.
private final Map<String, MQTTConnection> connectedClients;
private final Map<String, MQTTSessionState> sessionStates;
private final Map<String, MQTTConnection> connectedClients = new ConcurrentHashMap<>();
private final Map<String, MQTTSessionState> sessionStates = new ConcurrentHashMap<>();
private int defaultMqttSessionExpiryInterval = -1;
private final MQTTRedirectHandler redirectHandler;
MQTTProtocolManager(ActiveMQServer server,
Map<String, MQTTConnection> connectedClients,
Map<String, MQTTSessionState> sessionStates,
List<BaseInterceptor> incomingInterceptors,
List<BaseInterceptor> outgoingInterceptors) {
this.server = server;
this.connectedClients = connectedClients;
this.sessionStates = sessionStates;
this.updateInterceptors(incomingInterceptors, outgoingInterceptors);
server.getManagementService().addNotificationListener(this);
redirectHandler = new MQTTRedirectHandler(server);
}
public int getDefaultMqttSessionExpiryInterval() {
return defaultMqttSessionExpiryInterval;
}
public MQTTProtocolManager setDefaultMqttSessionExpiryInterval(int sessionExpiryInterval) {
this.defaultMqttSessionExpiryInterval = sessionExpiryInterval;
return this;
}
@Override
public void onNotification(Notification notification) {
if (!(notification.getType() instanceof CoreNotificationType))
@ -125,6 +132,25 @@ public class MQTTProtocolManager extends AbstractProtocolManager<MqttMessage, MQ
this.outgoingInterceptors.addAll(getFactory().filterInterceptors(outgoing));
}
public void scanSessions() {
if (defaultMqttSessionExpiryInterval == -1) {
log.debug("sessionExpiryInterval is -1 so skipping check");
} else {
for (Map.Entry<String, MQTTSessionState> entry : sessionStates.entrySet()) {
MQTTSessionState state = entry.getValue();
if (log.isDebugEnabled()) {
log.debug("Inspecting session state: " + state);
}
if (!state.getAttached() && state.getDisconnectedTime() + (defaultMqttSessionExpiryInterval * 1000) < System.currentTimeMillis()) {
if (log.isDebugEnabled()) {
log.debug("Removing expired session state: " + state);
}
sessionStates.remove(entry.getKey());
}
}
}
}
@Override
public ConnectionEntry createConnectionEntry(Acceptor acceptorUsed, Connection connection) {
try {

View File

@ -19,9 +19,13 @@ package org.apache.activemq.artemis.core.protocol.mqtt;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.core.BaseInterceptor;
import org.apache.activemq.artemis.core.protocol.ProtocolHandler;
import org.apache.activemq.artemis.core.server.ActiveMQComponent;
import org.apache.activemq.artemis.core.server.ActiveMQScheduledComponent;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.spi.core.protocol.AbstractProtocolManagerFactory;
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
@ -38,16 +42,14 @@ public class MQTTProtocolManagerFactory extends AbstractProtocolManagerFactory<M
private static final String[] SUPPORTED_PROTOCOLS = {MQTT_PROTOCOL_NAME};
private final Map<String, MQTTConnection> connectedClients = new ConcurrentHashMap<>();
private final Map<String, MQTTSessionState> sessionStates = new ConcurrentHashMap<>();
@Override
public ProtocolManager createProtocolManager(ActiveMQServer server,
final Map<String, Object> parameters,
List<BaseInterceptor> incomingInterceptors,
List<BaseInterceptor> outgoingInterceptors) throws Exception {
BeanSupport.stripPasswords(parameters);
return BeanSupport.setData(new MQTTProtocolManager(server, connectedClients, sessionStates, incomingInterceptors, outgoingInterceptors), parameters);
return BeanSupport.setData(new MQTTProtocolManager(server, incomingInterceptors, outgoingInterceptors), parameters);
}
@Override
@ -64,4 +66,31 @@ public class MQTTProtocolManagerFactory extends AbstractProtocolManagerFactory<M
public String getModuleName() {
return MODULE_NAME;
}
@Override
public void loadProtocolServices(ActiveMQServer server, List<ActiveMQComponent> services) {
services.add(new MQTTPeriodicTasks(server, server.getScheduledPool()));
}
public class MQTTPeriodicTasks extends ActiveMQScheduledComponent {
final ActiveMQServer server;
public MQTTPeriodicTasks(ActiveMQServer server, ScheduledExecutorService scheduledExecutorService) {
super(scheduledExecutorService, null, 5, TimeUnit.SECONDS, false);
this.server = server;
}
@Override
public void run() {
server.getRemotingService().getAcceptors().forEach((key, acceptor) -> {
ProtocolHandler protocolHandler = acceptor.getProtocolHandler();
if (protocolHandler != null) {
protocolHandler.getProtocolMap().values().forEach(m -> {
if (m instanceof MQTTProtocolManager) {
((MQTTProtocolManager)m).scanSessions();
}
});
}
});
}
}
}

View File

@ -108,6 +108,7 @@ public class MQTTSession {
if (state != null) {
state.setAttached(false);
state.setDisconnectedTime(System.currentTimeMillis());
}
if (isClean()) {
@ -178,6 +179,7 @@ public class MQTTSession {
void setSessionState(MQTTSessionState state) {
this.state = state;
state.setAttached(true);
this.state.setDisconnectedTime(0);
}
MQTTRetainMessageManager getRetainMessageManager() {

View File

@ -47,6 +47,8 @@ public class MQTTSessionState {
private boolean attached = false;
private long disconnectedTime = 0;
private final OutboundStore outboundStore = new OutboundStore();
public MQTTSessionState(String clientId) {
@ -59,6 +61,7 @@ public class MQTTSessionState {
addressMessageMap.clear();
pubRec.clear();
outboundStore.clear();
disconnectedTime = 0;
}
OutboundStore getOutboundStore() {
@ -120,6 +123,14 @@ public class MQTTSessionState {
this.clientId = clientId;
}
long getDisconnectedTime() {
return disconnectedTime;
}
void setDisconnectedTime(long disconnectedTime) {
this.disconnectedTime = disconnectedTime;
}
void removeMessageRef(Integer mqttId) {
MQTTMessageInfo info = messageRefStore.remove(mqttId);
if (info != null) {

View File

@ -82,6 +82,10 @@ public class ProtocolHandler {
}
}
public Map<String, ProtocolManager> getProtocolMap() {
return protocolMap;
}
public ChannelHandler getProtocolDecoder() {
return new ProtocolDecoder(true, false);
}

View File

@ -553,7 +553,7 @@ public class NettyAcceptor extends AbstractAcceptor {
return connections;
}
// Only for testing purposes
@Override
public ProtocolHandler getProtocolHandler() {
return protocolHandler;
}

View File

@ -118,6 +118,8 @@ public interface RemotingService {
*/
Acceptor getAcceptor(String name);
Map<String, Acceptor> getAcceptors();
Acceptor createAcceptor(String name, String uri) throws Exception;
Acceptor createAcceptor(TransportConfiguration transportConfiguration);

View File

@ -292,8 +292,7 @@ public class RemotingServiceImpl implements RemotingService, ServerConnectionLif
return acceptor;
}
/** No interface method, for tests only */
@Override
public Map<String, Acceptor> getAcceptors() {
return acceptors;
}

View File

@ -20,6 +20,7 @@ import java.util.List;
import java.util.Map;
import org.apache.activemq.artemis.api.core.BaseInterceptor;
import org.apache.activemq.artemis.core.protocol.ProtocolHandler;
import org.apache.activemq.artemis.core.security.ActiveMQPrincipal;
import org.apache.activemq.artemis.core.server.ActiveMQComponent;
import org.apache.activemq.artemis.core.server.cluster.ClusterConnection;
@ -78,4 +79,8 @@ public interface Acceptor extends ActiveMQComponent {
* stores on acceptors which support SSL.
*/
void reload();
default ProtocolHandler getProtocolHandler() {
return null;
}
}

View File

@ -150,3 +150,19 @@ MQTT over Web Sockets is supported via a normal MQTT acceptor:
With this configuration, Apache ActiveMQ Artemis will accept MQTT connections
over Web Sockets on the port `1883`. Web browsers can then connect to
`ws://<server>:1883` using a Web Socket to send and receive MQTT messages.
## Automatic Subscription Clean-up
Sometimes MQTT clients don't clean up their subscriptions. In such situations
the `auto-delete-queues-delay` and `auto-delete-queues-message-count`
address-settings can be used to clean up the abandoned subscription queues.
However, the MQTT session meta-data is still present in memory and needs to be
cleaned up as well. The URL parameter `defaultMqttSessionExpiryInterval` can be
configured on the MQTT `acceptor` to deal with this situation.
The default `defaultMqttSessionExpiryInterval` is `-1` which means no session
state will be expired. Otherwise it represents the number of _milliseconds_
which must elapse after the client has disconnected before the broker will
remove the session state.
MQTT session state is scanned every 5 seconds.

View File

@ -0,0 +1,54 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.apache.activemq.artemis.tests.integration.mqtt.imported;
import org.apache.activemq.artemis.tests.util.Wait;
import org.fusesource.mqtt.client.BlockingConnection;
import org.fusesource.mqtt.client.MQTT;
import org.fusesource.mqtt.client.QoS;
import org.fusesource.mqtt.client.Topic;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class MQTTSessionExpiryIntervalTest extends MQTTTestSupport {
private static final Logger log = LoggerFactory.getLogger(MQTTSessionExpiryIntervalTest.class);
@Test(timeout = 60 * 1000)
public void testCustomSessionExpiryInterval() throws Exception {
final MQTT mqttSub = createMQTTConnection("MQTT-Sub-Client", false);
BlockingConnection connectionSub = mqttSub.blockingConnection();
connectionSub.connect();
assertEquals(1, getSessions().size());
Topic[] topics = {new Topic("TopicA", QoS.EXACTLY_ONCE)};
connectionSub.subscribe(topics);
connectionSub.disconnect();
Wait.assertEquals(0, () -> getSessions().size(), 10000, 100);
}
@Override
protected void addMQTTConnector() throws Exception {
server.getConfiguration().addAcceptorConfiguration("MQTT", "tcp://localhost:" + port + "?protocols=MQTT;anycastPrefix=anycast:;multicastPrefix=multicast:;defaultMqttSessionExpiryInterval=3");
log.debug("Added MQTT connector to broker");
}
}

View File

@ -1601,6 +1601,23 @@ public class MQTTTest extends MQTTTestSupport {
assertEquals("Should have received " + (messagesPerRun * (numberOfRuns + 1)) + " messages", (messagesPerRun * (numberOfRuns + 1)), received);
}
@Test(timeout = 60 * 1000)
public void testDefaultSessionExpiryInterval() throws Exception {
final MQTT mqttSub = createMQTTConnection("MQTT-Sub-Client", false);
BlockingConnection connectionSub = mqttSub.blockingConnection();
connectionSub.connect();
assertEquals(1, getSessions().size());
Topic[] topics = {new Topic("TopicA", QoS.EXACTLY_ONCE)};
connectionSub.subscribe(topics);
connectionSub.disconnect();
// session shouldn't expire by default
Wait.assertEquals(1, () -> getSessions().size(), 10000, 100);
}
@Test(timeout = 30 * 1000)
public void testDefaultKeepAliveWhenClientSpecifiesZero() throws Exception {
stopBroker();