This commit is contained in:
Clebert Suconic 2021-08-06 08:34:07 -04:00
commit ce87777ead
200 changed files with 9049 additions and 356 deletions

View File

@ -25,7 +25,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import io.airlift.airline.Option;
import org.apache.activemq.artemis.api.core.client.ServerLocator;
import org.apache.activemq.artemis.api.core.management.ActiveMQManagementProxy;
import org.apache.activemq.artemis.cli.CLIException;
import org.apache.activemq.artemis.cli.commands.AbstractAction;
@ -72,10 +71,7 @@ public abstract class CheckAbstract extends AbstractAction {
int successTasks = 0;
try (ActiveMQConnectionFactory factory = createCoreConnectionFactory();
ServerLocator serverLocator = factory.getServerLocator();
ActiveMQManagementProxy managementProxy = new ActiveMQManagementProxy(serverLocator, user, password)) {
managementProxy.start();
ActiveMQManagementProxy managementProxy = new ActiveMQManagementProxy(factory.getServerLocator(), user, password)) {
StopWatch watch = new StopWatch();
CheckTask[] checkTasks = getCheckTasks();

View File

@ -61,7 +61,7 @@ public class CheckContext extends ActionContext {
public String getNodeId() throws Exception {
if (nodeId == null) {
nodeId = managementProxy.invokeOperation(String.class, "broker", "getNodeID");
nodeId = managementProxy.getAttribute("broker", "NodeID", String.class, 0);
}
return nodeId;
@ -69,8 +69,8 @@ public class CheckContext extends ActionContext {
public Map<String, NodeInfo> getTopology() throws Exception {
if (topology == null) {
topology = Arrays.stream(NodeInfo.from(managementProxy.invokeOperation(
String.class, "broker", "listNetworkTopology"))).
topology = Arrays.stream(NodeInfo.from((String)managementProxy.
invokeOperation("broker", "listNetworkTopology", null, null, 0))).
collect(Collectors.toMap(node -> node.getId(), node -> node));
}

View File

@ -138,7 +138,7 @@ public class NodeCheck extends CheckAbstract {
}
private void checkNodeUp(final CheckContext context) throws Exception {
if (!context.getManagementProxy().invokeOperation(Boolean.class, "broker", "isStarted")) {
if (!context.getManagementProxy().getAttribute("broker", "Started", Boolean.class, 0)) {
throw new CheckException("The node isn't started.");
}
}
@ -182,28 +182,31 @@ public class NodeCheck extends CheckAbstract {
}
private void checkNodeDiskUsage(final CheckContext context) throws Exception {
int thresholdValue;
Integer maxDiskUsage;
if (diskUsage == -1) {
thresholdValue = context.getManagementProxy().invokeOperation(
int.class, "broker", "getMaxDiskUsage");
maxDiskUsage = context.getManagementProxy().
getAttribute("broker", "MaxDiskUsage", Integer.class, 0);
} else {
thresholdValue = diskUsage;
maxDiskUsage = diskUsage;
}
checkNodeUsage(context, "getDiskStoreUsage", thresholdValue);
Double diskStoreUsage = context.getManagementProxy().
getAttribute("broker", "DiskStoreUsage", Double.class, 0);
checkNodeResourceUsage("DiskStoreUsage", (int)(diskStoreUsage * 100), maxDiskUsage);
}
private void checkNodeMemoryUsage(final CheckContext context) throws Exception {
checkNodeUsage(context, "getAddressMemoryUsagePercentage", memoryUsage);
int addressMemoryUsagePercentage = context.getManagementProxy().
getAttribute("broker", "AddressMemoryUsagePercentage", Integer.class, 0);
checkNodeResourceUsage("MemoryUsage", addressMemoryUsagePercentage, memoryUsage);
}
private void checkNodeUsage(final CheckContext context, final String name, final int thresholdValue) throws Exception {
int usageValue = context.getManagementProxy().invokeOperation(int.class, "broker", name);
private void checkNodeResourceUsage(final String resourceName, final int usageValue, final int thresholdValue) throws Exception {
if (usageValue > thresholdValue) {
throw new CheckException("The " + (name.startsWith("get") ? name.substring(3) : name) +
" " + usageValue + " is less than " + thresholdValue);
throw new CheckException("The " + resourceName + " " + usageValue + " is less than " + thresholdValue);
}
}
}

View File

@ -132,7 +132,7 @@ public class QueueCheck extends CheckAbstract {
}
private void checkQueueUp(final CheckContext context) throws Exception {
if (context.getManagementProxy().invokeOperation(Boolean.class,ResourceNames.QUEUE + getName(), "isPaused")) {
if (context.getManagementProxy().getAttribute(ResourceNames.QUEUE + getName(), "Paused", Boolean.class, 0)) {
throw new CheckException("The queue is paused.");
}
}

View File

@ -267,6 +267,12 @@ public enum ActiveMQExceptionType {
public ActiveMQException createException(String msg) {
return new ActiveMQDivertDoesNotExistException(msg);
}
},
REDIRECTED(222) {
@Override
public ActiveMQException createException(String msg) {
return new ActiveMQRedirectedException(msg);
}
};
private static final Map<Integer, ActiveMQExceptionType> TYPE_MAP;

View File

@ -0,0 +1,34 @@
/*
* 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
*
* 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.apache.activemq.artemis.api.core;
/**
* A client was redirected.
*/
public final class ActiveMQRedirectedException extends ActiveMQException {
private static final long serialVersionUID = 7414966383933311627L;
public ActiveMQRedirectedException() {
super(ActiveMQExceptionType.REDIRECTED);
}
public ActiveMQRedirectedException(String message) {
super(ActiveMQExceptionType.REDIRECTED, message);
}
}

View File

@ -0,0 +1,73 @@
/*
* 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
*
* 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.apache.activemq.artemis.api.core;
public enum DisconnectReason {
REDIRECT((byte)0, false),
REDIRECT_ON_CRITICAL_ERROR((byte)1, true),
SCALE_DOWN((byte)2, false),
SCALE_DOWN_ON_CRITICAL_ERROR((byte)3, true),
SHOUT_DOWN((byte)4, false),
SHOUT_DOWN_ON_CRITICAL_ERROR((byte)5, true);
private final byte type;
private final boolean criticalError;
DisconnectReason(byte type, boolean criticalError) {
this.type = type;
this.criticalError = criticalError;
}
public byte getType() {
return type;
}
public boolean isCriticalError() {
return criticalError;
}
public boolean isRedirect() {
return this == REDIRECT || this == REDIRECT_ON_CRITICAL_ERROR;
}
public boolean isScaleDown() {
return this == SCALE_DOWN || this == SCALE_DOWN_ON_CRITICAL_ERROR;
}
public boolean isShutDown() {
return this == SHOUT_DOWN || this == SHOUT_DOWN_ON_CRITICAL_ERROR;
}
public static DisconnectReason getType(byte type) {
switch (type) {
case 0:
return REDIRECT;
case 1:
return REDIRECT_ON_CRITICAL_ERROR;
case 2:
return SCALE_DOWN;
case 3:
return SCALE_DOWN_ON_CRITICAL_ERROR;
case 4:
return SHOUT_DOWN;
case 5:
return SHOUT_DOWN_ON_CRITICAL_ERROR;
default:
return null;
}
}
}

View File

@ -255,7 +255,7 @@ public class TransportConfiguration implements Serializable {
public String toString() {
StringBuilder str = new StringBuilder(TransportConfiguration.class.getSimpleName());
str.append("(name=" + name + ", ");
str.append("factory=" + replaceWildcardChars(factoryClassName));
str.append("factory=" + (factoryClassName == null ? "null" : replaceWildcardChars(factoryClassName)));
str.append(") ");
str.append(toStringParameters(params, extraProps));
return str.toString();

View File

@ -39,6 +39,14 @@ public interface ClientSession extends XAResource, AutoCloseable {
*/
String JMS_SESSION_IDENTIFIER_PROPERTY = "jms-session";
/**
* Just like {@link ClientSession.AddressQuery#JMS_SESSION_IDENTIFIER_PROPERTY} this is
* used to identify the ClientID over JMS Session.
* However this is only used when the JMS Session.clientID is set (which is optional).
* With this property management tools and the server can identify the jms-client-id used over JMS
*/
String JMS_SESSION_CLIENT_ID_PROPERTY = "jms-client-id";
/**
* Information returned by a binding query
*

View File

@ -135,6 +135,32 @@ public interface ClientSessionFactory extends AutoCloseable {
boolean preAcknowledge,
int ackBatchSize) throws ActiveMQException;
/**
* Creates an <em>authenticated</em> session.
* <p>
* It is possible to <em>pre-acknowledge messages on the server</em> so that the client can avoid additional network trip
* to the server to acknowledge messages. While this increase performance, this does not guarantee delivery (as messages
* can be lost after being pre-acknowledged on the server). Use with caution if your application design permits it.
*
* @param username the user name
* @param password the user password
* @param xa whether the session support XA transaction semantic or not
* @param autoCommitSends <code>true</code> to automatically commit message sends, <code>false</code> to commit manually
* @param autoCommitAcks <code>true</code> to automatically commit message acknowledgement, <code>false</code> to commit manually
* @param preAcknowledge <code>true</code> to pre-acknowledge messages on the server, <code>false</code> to let the client acknowledge the messages
* @param clientID the session clientID
* @return a ClientSession
* @throws ActiveMQException if an exception occurs while creating the session
*/
ClientSession createSession(String username,
String password,
boolean xa,
boolean autoCommitSends,
boolean autoCommitAcks,
boolean preAcknowledge,
int ackBatchSize,
String clientID) throws ActiveMQException;
/**
* Closes this factory and any session created by it.
*/

View File

@ -18,7 +18,6 @@
package org.apache.activemq.artemis.api.core.management;
import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
import org.apache.activemq.artemis.api.core.client.ClientMessage;
import org.apache.activemq.artemis.api.core.client.ClientRequestor;
@ -28,51 +27,67 @@ import org.apache.activemq.artemis.api.core.client.ServerLocator;
public class ActiveMQManagementProxy implements AutoCloseable {
private final String username;
private final String password;
private final ServerLocator locator;
private final ServerLocator serverLocator;
private ClientSessionFactory sessionFactory;
private ClientSession session;
private ClientRequestor requestor;
private final ClientSessionFactory sessionFactory;
public ActiveMQManagementProxy(final ServerLocator locator, final String username, final String password) {
this.locator = locator;
this.username = username;
this.password = password;
private final ClientSession clientSession;
public ActiveMQManagementProxy(final ClientSession session) {
serverLocator = null;
sessionFactory = null;
clientSession = session;
}
public void start() throws Exception {
public ActiveMQManagementProxy(final ServerLocator locator, final String username, final String password) throws Exception {
serverLocator = locator;
sessionFactory = locator.createSessionFactory();
session = sessionFactory.createSession(username, password, false, true, true, false, ActiveMQClient.DEFAULT_ACK_BATCH_SIZE);
requestor = new ClientRequestor(session, ActiveMQDefaultConfiguration.getDefaultManagementAddress());
session.start();
clientSession = sessionFactory.createSession(username, password, false, true, true, false, ActiveMQClient.DEFAULT_ACK_BATCH_SIZE).start();
}
public <T> T invokeOperation(final Class<T> type, final String resourceName, final String operationName, final Object... operationArgs) throws Exception {
ClientMessage request = session.createMessage(false);
public <T> T getAttribute(final String resourceName, final String attributeName, final Class<T> attributeClass, final int timeout) throws Exception {
try (ClientRequestor requestor = new ClientRequestor(clientSession, ActiveMQDefaultConfiguration.getDefaultManagementAddress())) {
ClientMessage request = clientSession.createMessage(false);
ManagementHelper.putOperationInvocation(request, resourceName, operationName, operationArgs);
ManagementHelper.putAttribute(request, resourceName, attributeName);
ClientMessage reply = requestor.request(request);
ClientMessage reply = requestor.request(request, timeout);
if (ManagementHelper.hasOperationSucceeded(reply)) {
return (T)ManagementHelper.getResult(reply, type);
} else {
throw new Exception("Failed to invoke " + resourceName + "." + operationName + ". Reason: " + ManagementHelper.getResult(reply, String.class));
if (ManagementHelper.hasOperationSucceeded(reply)) {
return (T)ManagementHelper.getResult(reply, attributeClass);
} else {
throw new Exception("Failed to get " + resourceName + "." + attributeName + ". Reason: " + ManagementHelper.getResult(reply, String.class));
}
}
}
public <T> T invokeOperation(final String resourceName, final String operationName, final Object[] operationParams, final Class<T> operationClass, final int timeout) throws Exception {
try (ClientRequestor requestor = new ClientRequestor(clientSession, ActiveMQDefaultConfiguration.getDefaultManagementAddress())) {
ClientMessage request = clientSession.createMessage(false);
public void stop() throws ActiveMQException {
session.stop();
ManagementHelper.putOperationInvocation(request, resourceName, operationName, operationParams);
ClientMessage reply = requestor.request(request, timeout);
if (ManagementHelper.hasOperationSucceeded(reply)) {
return (T)ManagementHelper.getResult(reply, operationClass);
} else {
throw new Exception("Failed to invoke " + resourceName + "." + operationName + ". Reason: " + ManagementHelper.getResult(reply, String.class));
}
}
}
@Override
public void close() throws Exception {
requestor.close();
session.close();
sessionFactory.close();
clientSession.close();
if (sessionFactory != null) {
sessionFactory.close();
}
if (serverLocator != null) {
serverLocator.close();
}
}
}

View File

@ -0,0 +1,31 @@
/*
* 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
*
* 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.apache.activemq.artemis.api.core.management;
import javax.management.MBeanOperationInfo;
import javax.management.openmbean.CompositeData;
/**
* A BrokerBalancerControl is used to manage a BrokerBalancer.
*/
public interface BrokerBalancerControl {
@Operation(desc = "Get the target associated with key", impact = MBeanOperationInfo.INFO)
CompositeData getTarget(@Parameter(desc = "a key", name = "key") String key) throws Exception;
@Operation(desc = "Get the target associated with key as JSON", impact = MBeanOperationInfo.INFO)
String getTargetAsJSON(@Parameter(desc = "a key", name = "key") String key);
}

View File

@ -149,6 +149,15 @@ public final class ObjectNameBuilder {
return createObjectName("cluster-connection", name);
}
/**
* Returns the ObjectName used by BrokerBalancerControl.
*
* @see BrokerBalancerControl
*/
public ObjectName getBrokerBalancerObjectName(final String name) throws Exception {
return createObjectName("broker-balancer", name);
}
private ObjectName createObjectName(final String type, final String name) throws Exception {
return ObjectName.getInstance(String.format("%s,component=%ss,name=%s", getActiveMQServerName(), type, ObjectName.quote(name)));
}

View File

@ -44,6 +44,8 @@ public final class ResourceNames {
public static final String BROADCAST_GROUP = "broadcastgroup.";
public static final String BROKER_BALANCER = "brokerbalancer.";
public static final String RETROACTIVE_SUFFIX = "retro";
public static SimpleString getRetroactiveResourceQueueName(String prefix, String delimiter, SimpleString address, RoutingType routingType) {

View File

@ -27,6 +27,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQLargeMessageException;
import org.apache.activemq.artemis.api.core.ActiveMQLargeMessageInterruptedException;
import org.apache.activemq.artemis.api.core.ActiveMQNotConnectedException;
import org.apache.activemq.artemis.api.core.ActiveMQObjectClosedException;
import org.apache.activemq.artemis.api.core.ActiveMQRedirectedException;
import org.apache.activemq.artemis.api.core.ActiveMQTransactionOutcomeUnknownException;
import org.apache.activemq.artemis.api.core.ActiveMQTransactionRolledBackException;
import org.apache.activemq.artemis.api.core.ActiveMQUnBlockedException;
@ -237,4 +238,7 @@ public interface ActiveMQClientMessageBundle {
@Message(id = 219065, value = "Failed to handle packet.")
RuntimeException failedToHandlePacket(@Cause Exception e);
@Message(id = 219066, value = "The connection was redirected")
ActiveMQRedirectedException redirected();
}

View File

@ -37,6 +37,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.ActiveMQInterruptedException;
import org.apache.activemq.artemis.api.core.ActiveMQNotConnectedException;
import org.apache.activemq.artemis.api.core.DisconnectReason;
import org.apache.activemq.artemis.api.core.Interceptor;
import org.apache.activemq.artemis.api.core.Pair;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
@ -79,11 +80,13 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
private final ClientProtocolManager clientProtocolManager;
private TransportConfiguration connectorConfig;
private final TransportConfiguration connectorConfig;
private TransportConfiguration currentConnectorConfig;
private TransportConfiguration previousConnectorConfig;
private volatile TransportConfiguration backupConfig;
private volatile TransportConfiguration currentConnectorConfig;
private volatile TransportConfiguration backupConnectorConfig;
private ConnectorFactory connectorFactory;
@ -184,6 +187,8 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
this.clientProtocolManager.setSessionFactory(this);
this.connectorConfig = connectorConfig.getA();
this.currentConnectorConfig = connectorConfig.getA();
connectorFactory = instantiateConnectorFactory(connectorConfig.getA().getFactoryClassName());
@ -231,7 +236,7 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
connectionReadyForWrites = true;
if (connectorConfig.getB() != null) {
this.backupConfig = connectorConfig.getB();
this.backupConnectorConfig = connectorConfig.getB();
}
}
@ -253,8 +258,8 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
if (connection == null) {
StringBuilder msg = new StringBuilder("Unable to connect to server using configuration ").append(currentConnectorConfig);
if (backupConfig != null) {
msg.append(" and backup configuration ").append(backupConfig);
if (backupConnectorConfig != null) {
msg.append(" and backup configuration ").append(backupConnectorConfig);
}
throw new ActiveMQNotConnectedException(msg.toString());
}
@ -288,7 +293,7 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
if (logger.isDebugEnabled()) {
logger.debug("Setting up backup config = " + backUp + " for live = " + live);
}
backupConfig = backUp;
backupConnectorConfig = backUp;
} else {
if (logger.isDebugEnabled()) {
logger.debug("ClientSessionFactoryImpl received backup update for live/backup pair = " + live +
@ -302,7 +307,19 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
@Override
public Object getBackupConnector() {
return backupConfig;
return backupConnectorConfig;
}
@Override
public ClientSession createSession(final String username,
final String password,
final boolean xa,
final boolean autoCommitSends,
final boolean autoCommitAcks,
final boolean preAcknowledge,
final int ackBatchSize,
final String clientID) throws ActiveMQException {
return createSessionInternal(username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, ackBatchSize, clientID);
}
@Override
@ -313,42 +330,42 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
final boolean autoCommitAcks,
final boolean preAcknowledge,
final int ackBatchSize) throws ActiveMQException {
return createSessionInternal(username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, ackBatchSize);
return createSessionInternal(username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, ackBatchSize, null);
}
@Override
public ClientSession createSession(final boolean autoCommitSends,
final boolean autoCommitAcks,
final int ackBatchSize) throws ActiveMQException {
return createSessionInternal(null, null, false, autoCommitSends, autoCommitAcks, serverLocator.isPreAcknowledge(), ackBatchSize);
return createSessionInternal(null, null, false, autoCommitSends, autoCommitAcks, serverLocator.isPreAcknowledge(), ackBatchSize, null);
}
@Override
public ClientSession createXASession() throws ActiveMQException {
return createSessionInternal(null, null, true, false, false, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize());
return createSessionInternal(null, null, true, false, false, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize(), null);
}
@Override
public ClientSession createTransactedSession() throws ActiveMQException {
return createSessionInternal(null, null, false, false, false, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize());
return createSessionInternal(null, null, false, false, false, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize(), null);
}
@Override
public ClientSession createSession() throws ActiveMQException {
return createSessionInternal(null, null, false, true, true, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize());
return createSessionInternal(null, null, false, true, true, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize(), null);
}
@Override
public ClientSession createSession(final boolean autoCommitSends,
final boolean autoCommitAcks) throws ActiveMQException {
return createSessionInternal(null, null, false, autoCommitSends, autoCommitAcks, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize());
return createSessionInternal(null, null, false, autoCommitSends, autoCommitAcks, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize(), null);
}
@Override
public ClientSession createSession(final boolean xa,
final boolean autoCommitSends,
final boolean autoCommitAcks) throws ActiveMQException {
return createSessionInternal(null, null, xa, autoCommitSends, autoCommitAcks, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize());
return createSessionInternal(null, null, xa, autoCommitSends, autoCommitAcks, serverLocator.isPreAcknowledge(), serverLocator.getAckBatchSize(), null);
}
@Override
@ -356,7 +373,7 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
final boolean autoCommitSends,
final boolean autoCommitAcks,
final boolean preAcknowledge) throws ActiveMQException {
return createSessionInternal(null, null, xa, autoCommitSends, autoCommitAcks, preAcknowledge, serverLocator.getAckBatchSize());
return createSessionInternal(null, null, xa, autoCommitSends, autoCommitAcks, preAcknowledge, serverLocator.getAckBatchSize(), null);
}
// ClientConnectionLifeCycleListener implementation --------------------------------------------------
@ -632,10 +649,19 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
connector = null;
boolean allSessionReconnected;
HashSet<ClientSessionInternal> sessionsToFailover;
synchronized (sessions) {
sessionsToFailover = new HashSet<>(sessions);
}
for (ClientSessionInternal session : sessionsToFailover) {
session.preHandleFailover(connection);
}
boolean allSessionReconnected = false;
int failedReconnectSessionsCounter = 0;
do {
allSessionReconnected = reconnectSessions(oldConnection, reconnectAttempts, me);
allSessionReconnected = reconnectSessions(sessionsToFailover, oldConnection, reconnectAttempts, me);
if (oldConnection != null) {
oldConnection.destroy();
}
@ -644,10 +670,19 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
failedReconnectSessionsCounter++;
oldConnection = connection;
connection = null;
// Wait for retry when the connection is established but not all session are reconnected.
if ((reconnectAttempts == -1 || failedReconnectSessionsCounter < reconnectAttempts) && oldConnection != null) {
waitForRetry(retryInterval);
}
}
}
while ((reconnectAttempts == -1 || failedReconnectSessionsCounter < reconnectAttempts) && !allSessionReconnected);
for (ClientSessionInternal session : sessionsToFailover) {
session.postHandleFailover(connection, allSessionReconnected);
}
if (oldConnection != null) {
oldConnection.destroy();
}
@ -699,10 +734,11 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
final boolean autoCommitSends,
final boolean autoCommitAcks,
final boolean preAcknowledge,
final int ackBatchSize) throws ActiveMQException {
final int ackBatchSize,
final String clientID) throws ActiveMQException {
String name = UUIDGenerator.getInstance().generateStringUUID();
SessionContext context = createSessionChannel(name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge);
SessionContext context = createSessionChannel(name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, clientID);
ClientSessionInternal session = new ClientSessionImpl(this, name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, serverLocator.isBlockOnAcknowledge(), serverLocator.isAutoGroup(), ackBatchSize, serverLocator.getConsumerWindowSize(), serverLocator.getConsumerMaxRate(), serverLocator.getConfirmationWindowSize(), serverLocator.getProducerWindowSize(), serverLocator.getProducerMaxRate(), serverLocator.isBlockOnNonDurableSend(), serverLocator.isBlockOnDurableSend(), serverLocator.isCacheLargeMessagesClient(), serverLocator.getMinLargeMessageSize(), serverLocator.isCompressLargeMessage(), serverLocator.getInitialMessagePacketSize(), serverLocator.getGroupID(), context, orderedExecutorFactory.getExecutor(), orderedExecutorFactory.getExecutor(), orderedExecutorFactory.getExecutor());
@ -764,18 +800,10 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
/*
* Re-attach sessions all pre-existing sessions to the new remoting connection
*/
private boolean reconnectSessions(final RemotingConnection oldConnection,
final int reconnectAttempts,
final ActiveMQException cause) {
HashSet<ClientSessionInternal> sessionsToFailover;
synchronized (sessions) {
sessionsToFailover = new HashSet<>(sessions);
}
for (ClientSessionInternal session : sessionsToFailover) {
session.preHandleFailover(connection);
}
private boolean reconnectSessions(final Set<ClientSessionInternal> sessionsToFailover,
final RemotingConnection oldConnection,
final int reconnectAttempts,
final ActiveMQException cause) {
getConnectionWithRetry(reconnectAttempts, oldConnection);
if (connection == null) {
@ -1040,11 +1068,13 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
public class CloseRunnable implements Runnable {
private final RemotingConnection conn;
private final String scaleDownTargetNodeID;
private final DisconnectReason reason;
private final String targetNodeID;
public CloseRunnable(RemotingConnection conn, String scaleDownTargetNodeID) {
public CloseRunnable(RemotingConnection conn, DisconnectReason reason, String targetNodeID) {
this.conn = conn;
this.scaleDownTargetNodeID = scaleDownTargetNodeID;
this.reason = reason;
this.targetNodeID = targetNodeID;
}
// Must be executed on new thread since cannot block the Netty thread for a long time and fail
@ -1053,10 +1083,12 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
public void run() {
try {
CLOSE_RUNNABLES.add(this);
if (scaleDownTargetNodeID == null) {
conn.fail(ActiveMQClientMessageBundle.BUNDLE.disconnected());
if (reason.isRedirect()) {
conn.fail(ActiveMQClientMessageBundle.BUNDLE.redirected());
} else if (reason.isScaleDown()) {
conn.fail(ActiveMQClientMessageBundle.BUNDLE.disconnected(), targetNodeID);
} else {
conn.fail(ActiveMQClientMessageBundle.BUNDLE.disconnected(), scaleDownTargetNodeID);
conn.fail(ActiveMQClientMessageBundle.BUNDLE.disconnected());
}
} finally {
CLOSE_RUNNABLES.remove(this);
@ -1136,72 +1168,39 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
Connection transportConnection = null;
try {
if (logger.isDebugEnabled()) {
logger.debug("Trying to connect with connectorFactory = " + connectorFactory +
", connectorConfig=" + currentConnectorConfig);
//Try to connect with the current connector configuration
transportConnection = createTransportConnection("current", currentConnectorConfig);
if (transportConnection != null) {
return transportConnection;
}
Connector liveConnector = createConnector(connectorFactory, currentConnectorConfig);
if ((transportConnection = openTransportConnection(liveConnector)) != null) {
// if we can't connect the connect method will return null, hence we have to try the backup
connector = liveConnector;
return transportConnection;
} else if (backupConfig != null) {
if (logger.isDebugEnabled()) {
logger.debug("Trying backup config = " + backupConfig);
}
ConnectorFactory backupConnectorFactory = instantiateConnectorFactory(backupConfig.getFactoryClassName());
Connector backupConnector = createConnector(backupConnectorFactory, backupConfig);
transportConnection = openTransportConnection(backupConnector);
if (backupConnectorConfig != null) {
//Try to connect with the backup connector configuration
transportConnection = createTransportConnection("backup", backupConnectorConfig);
if (transportConnection != null) {
/*looks like the backup is now live, let's use that*/
if (logger.isDebugEnabled()) {
logger.debug("Connected to the backup at " + backupConfig);
}
// Switching backup as live
connector = backupConnector;
connectorConfig = currentConnectorConfig;
currentConnectorConfig = backupConfig;
connectorFactory = backupConnectorFactory;
return transportConnection;
}
}
if (logger.isDebugEnabled()) {
logger.debug("Backup is not active, trying original connection configuration now.");
if (previousConnectorConfig != null && !currentConnectorConfig.equals(previousConnectorConfig)) {
//Try to connect with the previous connector configuration
transportConnection = createTransportConnection("previous", previousConnectorConfig);
if (transportConnection != null) {
return transportConnection;
}
}
if (currentConnectorConfig.equals(connectorConfig) || connectorConfig == null) {
// There was no changes on current and original connectors, just return null here and let the retry happen at the first portion of this method on the next retry
return null;
if (!currentConnectorConfig.equals(connectorConfig)) {
//Try to connect with the initial connector configuration
transportConnection = createTransportConnection("initial", connectorConfig);
if (transportConnection != null) {
return transportConnection;
}
}
ConnectorFactory lastConnectorFactory = instantiateConnectorFactory(connectorConfig.getFactoryClassName());
Connector lastConnector = createConnector(lastConnectorFactory, connectorConfig);
transportConnection = openTransportConnection(lastConnector);
if (transportConnection != null) {
logger.debug("Returning into original connector");
connector = lastConnector;
TransportConfiguration temp = currentConnectorConfig;
currentConnectorConfig = connectorConfig;
connectorConfig = temp;
return transportConnection;
} else {
logger.debug("no connection been made, returning null");
return null;
}
logger.debug("no connection been made, returning null");
return null;
} catch (Exception cause) {
// Sanity catch for badly behaved remoting plugins
@ -1226,6 +1225,33 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
}
private Connection createTransportConnection(String name, TransportConfiguration transportConnectorConfig) {
ConnectorFactory transportConnectorFactory = instantiateConnectorFactory(
transportConnectorConfig.getFactoryClassName());
if (logger.isDebugEnabled()) {
logger.debug("Trying to connect with connectorFactory=" + transportConnectorFactory
+ " and " + name + "ConnectorConfig: " + transportConnectorConfig);
}
Connector transportConnector = createConnector(transportConnectorFactory, transportConnectorConfig);
Connection transportConnection = openTransportConnection(transportConnector);
if (transportConnection != null) {
if (logger.isDebugEnabled()) {
logger.debug("Connected with the " + name + "ConnectorConfig=" + transportConnectorConfig);
}
connector = transportConnector;
connectorFactory = transportConnectorFactory;
previousConnectorConfig = currentConnectorConfig;
currentConnectorConfig = transportConnectorConfig;
}
return transportConnection;
}
private class DelegatingBufferHandler implements BufferHandler {
@Override
@ -1403,9 +1429,10 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
final boolean xa,
final boolean autoCommitSends,
final boolean autoCommitAcks,
final boolean preAcknowledge) throws ActiveMQException {
final boolean preAcknowledge,
final String clientID) throws ActiveMQException {
synchronized (createSessionLock) {
return clientProtocolManager.createSessionContext(name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, serverLocator.getMinLargeMessageSize(), serverLocator.getConfirmationWindowSize());
return clientProtocolManager.createSessionContext(name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, serverLocator.getMinLargeMessageSize(), serverLocator.getConfirmationWindowSize(), clientID);
}
}
@ -1417,20 +1444,39 @@ public class ClientSessionFactoryImpl implements ClientSessionFactoryInternal, C
class SessionFactoryTopologyHandler implements TopologyResponseHandler {
@Override
public void nodeDisconnected(RemotingConnection conn, String nodeID, String scaleDownTargetNodeID) {
public void nodeDisconnected(RemotingConnection conn, String nodeID, DisconnectReason reason, String targetNodeID, TransportConfiguration tagetConnector) {
if (logger.isTraceEnabled()) {
logger.trace("Disconnect being called on client:" +
" server locator = " +
serverLocator +
" notifying node " +
nodeID +
" as down", new Exception("trace"));
" server locator = " +
serverLocator +
" notifying node " +
nodeID +
" as down with reason " +
reason, new Exception("trace"));
}
serverLocator.notifyNodeDown(System.currentTimeMillis(), nodeID);
closeExecutor.execute(new CloseRunnable(conn, scaleDownTargetNodeID));
if (reason.isRedirect()) {
if (serverLocator.isHA()) {
TopologyMemberImpl topologyMember = serverLocator.getTopology().getMember(nodeID);
if (topologyMember != null) {
if (topologyMember.getConnector().getB() != null) {
backupConnectorConfig = topologyMember.getConnector().getB();
} else if (logger.isDebugEnabled()) {
logger.debug("The topology member " + nodeID + " with connector " + tagetConnector + " has no backup");
}
} else if (logger.isDebugEnabled()) {
logger.debug("The topology member " + nodeID + " with connector " + tagetConnector + " not found");
}
}
currentConnectorConfig = tagetConnector;
}
closeExecutor.execute(new CloseRunnable(conn, reason, targetNodeID));
}

View File

@ -33,6 +33,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
import org.apache.activemq.artemis.api.core.ActiveMQRedirectedException;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.QueueAttributes;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
@ -141,6 +142,8 @@ public final class ClientSessionImpl implements ClientSessionInternal, FailureLi
private volatile boolean mayAttemptToFailover = true;
private volatile boolean resetCreditManager = false;
/**
* Current XID. this will be used in case of failover
*/
@ -1387,8 +1390,6 @@ public final class ClientSessionImpl implements ClientSessionInternal, FailureLi
return true;
}
boolean resetCreditManager = false;
try {
// TODO remove this and encapsulate it
@ -1460,33 +1461,48 @@ public final class ClientSessionImpl implements ClientSessionInternal, FailureLi
sessionContext.returnBlocking(cause);
}
} catch (ActiveMQRedirectedException e) {
logger.info("failedToHandleFailover.ActiveMQRedirectedException");
suc = false;
} catch (Throwable t) {
ActiveMQClientLogger.LOGGER.failedToHandleFailover(t);
suc = false;
} finally {
sessionContext.releaseCommunications();
}
if (resetCreditManager) {
synchronized (producerCreditManager) {
producerCreditManager.reset();
}
// Also need to send more credits for consumers, otherwise the system could hand with the server
// not having any credits to send
}
}
HashMap<String, String> metaDataToSend;
synchronized (metadata) {
metaDataToSend = new HashMap<>(metadata);
}
sessionContext.resetMetadata(metaDataToSend);
return suc;
}
@Override
public void postHandleFailover(RemotingConnection connection, boolean successful) {
sessionContext.releaseCommunications();
if (successful) {
synchronized (this) {
if (closed) {
return;
}
if (resetCreditManager) {
synchronized (producerCreditManager) {
producerCreditManager.reset();
}
resetCreditManager = false;
// Also need to send more credits for consumers, otherwise the system could hand with the server
// not having any credits to send
}
}
HashMap<String, String> metaDataToSend;
synchronized (metadata) {
metaDataToSend = new HashMap<>(metadata);
}
sessionContext.resetMetadata(metaDataToSend);
}
}
@Override

View File

@ -68,6 +68,8 @@ public interface ClientSessionInternal extends ClientSession {
boolean handleFailover(RemotingConnection backupConnection, ActiveMQException cause);
void postHandleFailover(RemotingConnection connection, boolean successful);
RemotingConnection getConnection();
void cleanUp(boolean failingOver) throws ActiveMQException;

View File

@ -184,6 +184,13 @@ public interface Channel {
*/
int getLastConfirmedCommandID();
/**
* queries if this channel is locked. This method is designed for use in monitoring of the system state, not for synchronization control.
*
* @return true it the channel is locked and false otherwise
*/
boolean isLocked();
/**
* locks the channel.
* <p>

View File

@ -51,6 +51,16 @@ public interface CoreRemotingConnection extends RemotingConnection {
return version >= PacketImpl.ARTEMIS_2_7_0_VERSION;
}
default boolean isVersionSupportClientID() {
int version = getChannelVersion();
return version >= PacketImpl.ARTEMIS_2_18_0_VERSION;
}
default boolean isVersionSupportRedirect() {
int version = getChannelVersion();
return version >= PacketImpl.ARTEMIS_2_18_0_VERSION;
}
/**
* Sets the client protocol used on the communication. This will determine if the client has
* support for certain packet types

View File

@ -27,6 +27,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
import org.apache.activemq.artemis.api.core.ActiveMQInterruptedException;
import org.apache.activemq.artemis.api.core.DisconnectReason;
import org.apache.activemq.artemis.api.core.Interceptor;
import org.apache.activemq.artemis.api.core.Pair;
import org.apache.activemq.artemis.api.core.SimpleString;
@ -45,10 +46,13 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CheckFailo
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V3;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V4;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionResponseMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage_V3;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.Ping;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SubscribeClusterTopologyUpdatesMessageV2;
import org.apache.activemq.artemis.core.remoting.impl.netty.ActiveMQFrameDecoder2;
@ -243,10 +247,11 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
boolean autoCommitAcks,
boolean preAcknowledge,
int minLargeMessageSize,
int confirmationWindowSize) throws ActiveMQException {
int confirmationWindowSize,
String clientID) throws ActiveMQException {
for (Version clientVersion : VersionLoader.getClientVersions()) {
try {
return createSessionContext(clientVersion, name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, minLargeMessageSize, confirmationWindowSize);
return createSessionContext(clientVersion, name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, minLargeMessageSize, confirmationWindowSize, clientID);
} catch (ActiveMQException e) {
if (e.getType() != ActiveMQExceptionType.INCOMPATIBLE_CLIENT_SERVER_VERSIONS) {
throw e;
@ -266,7 +271,8 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
boolean autoCommitAcks,
boolean preAcknowledge,
int minLargeMessageSize,
int confirmationWindowSize) throws ActiveMQException {
int confirmationWindowSize,
String clientID) throws ActiveMQException {
if (!isAlive())
throw ActiveMQClientMessageBundle.BUNDLE.clientSessionClosed();
@ -293,7 +299,7 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
long sessionChannelID = connection.generateChannelID();
Packet request = newCreateSessionPacket(clientVersion, name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, minLargeMessageSize, confirmationWindowSize, sessionChannelID);
Packet request = newCreateSessionPacket(clientVersion.getIncrementingVersion(), name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, minLargeMessageSize, confirmationWindowSize, sessionChannelID, clientID);
try {
// channel1 reference here has to go away
@ -302,7 +308,8 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
if (!isAlive())
throw cause;
if (cause.getType() == ActiveMQExceptionType.UNBLOCKED) {
if (cause.getType() == ActiveMQExceptionType.UNBLOCKED ||
cause.getType() == ActiveMQExceptionType.REDIRECTED) {
// This means the thread was blocked on create session and failover unblocked it
// so failover could occur
@ -339,11 +346,11 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
}
while (retry);
sessionChannel.getConnection().setChannelVersion(response.getServerVersion());
return newSessionContext(name, confirmationWindowSize, sessionChannel, response);
return newSessionContext(name, confirmationWindowSize, sessionChannel, response);
}
protected Packet newCreateSessionPacket(Version clientVersion,
protected Packet newCreateSessionPacket(int clientVersion,
String name,
String username,
String password,
@ -353,8 +360,13 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
boolean preAcknowledge,
int minLargeMessageSize,
int confirmationWindowSize,
long sessionChannelID) {
return new CreateSessionMessage(name, sessionChannelID, clientVersion.getIncrementingVersion(), username, password, minLargeMessageSize, xa, autoCommitSends, autoCommitAcks, preAcknowledge, confirmationWindowSize, null);
long sessionChannelID,
String clientID) {
if (connection.isVersionSupportClientID()) {
return new CreateSessionMessage_V2(name, sessionChannelID, clientVersion, username, password, minLargeMessageSize, xa, autoCommitSends, autoCommitAcks, preAcknowledge, confirmationWindowSize, null, clientID);
} else {
return new CreateSessionMessage(name, sessionChannelID, clientVersion, username, password, minLargeMessageSize, xa, autoCommitSends, autoCommitAcks, preAcknowledge, confirmationWindowSize, null);
}
}
protected SessionContext newSessionContext(String name,
@ -459,19 +471,15 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
public void handlePacket(final Packet packet) {
final byte type = packet.getType();
if (type == PacketImpl.DISCONNECT || type == PacketImpl.DISCONNECT_V2) {
final DisconnectMessage msg = (DisconnectMessage) packet;
String scaleDownTargetNodeID = null;
SimpleString nodeID = msg.getNodeID();
if (packet instanceof DisconnectMessage_V2) {
final DisconnectMessage_V2 msg_v2 = (DisconnectMessage_V2) packet;
scaleDownTargetNodeID = msg_v2.getScaleDownNodeID() == null ? null : msg_v2.getScaleDownNodeID().toString();
}
if (topologyResponseHandler != null)
topologyResponseHandler.nodeDisconnected(conn, nodeID == null ? null : nodeID.toString(), scaleDownTargetNodeID);
if (type == PacketImpl.DISCONNECT) {
final DisconnectMessage disMessage = (DisconnectMessage) packet;
handleDisconnect(disMessage.getNodeID(), null, null, null);
} else if (type == PacketImpl.DISCONNECT_V2) {
final DisconnectMessage_V2 disMessage = (DisconnectMessage_V2) packet;
handleDisconnect(disMessage.getNodeID(), DisconnectReason.SCALE_DOWN, disMessage.getScaleDownNodeID(), null);
} else if (type == PacketImpl.DISCONNECT_V3) {
final DisconnectMessage_V3 disMessage = (DisconnectMessage_V3) packet;
handleDisconnect(disMessage.getNodeID(), disMessage.getReason(), disMessage.getTargetNodeID(), disMessage.getTargetConnector());
} else if (type == PacketImpl.CLUSTER_TOPOLOGY) {
ClusterTopologyChangeMessage topMessage = (ClusterTopologyChangeMessage) packet;
notifyTopologyChange(updateTransportConfiguration(topMessage));
@ -481,11 +489,22 @@ public class ActiveMQClientProtocolManager implements ClientProtocolManager {
} else if (type == PacketImpl.CLUSTER_TOPOLOGY_V3) {
ClusterTopologyChangeMessage_V3 topMessage = (ClusterTopologyChangeMessage_V3) packet;
notifyTopologyChange(updateTransportConfiguration(topMessage));
} else if (type == PacketImpl.CLUSTER_TOPOLOGY_V4) {
ClusterTopologyChangeMessage_V4 topMessage = (ClusterTopologyChangeMessage_V4) packet;
notifyTopologyChange(updateTransportConfiguration(topMessage));
connection.setChannelVersion(topMessage.getServerVersion());
} else if (type == PacketImpl.CHECK_FOR_FAILOVER_REPLY) {
System.out.println("Channel0Handler.handlePacket");
}
}
private void handleDisconnect(SimpleString nodeID, DisconnectReason reason, SimpleString targetNodeID, TransportConfiguration tagetConnector) {
if (topologyResponseHandler != null) {
topologyResponseHandler.nodeDisconnected(conn, nodeID == null ? null : nodeID.toString(), reason,
targetNodeID == null ? null : targetNodeID.toString(), tagetConnector);
}
}
/**
* @param topMessage
*/

View File

@ -139,7 +139,7 @@ public class ActiveMQSessionContext extends SessionContext {
private String name;
private boolean killed;
protected Channel getSessionChannel() {
public Channel getSessionChannel() {
return sessionChannel;
}

View File

@ -184,6 +184,10 @@ public final class ChannelImpl implements Channel {
return version >= 129;
case PacketImpl.SESS_BINDINGQUERY_RESP_V4:
return version >= 129;
case PacketImpl.CLUSTER_TOPOLOGY_V4:
case PacketImpl.CREATESESSION_V2:
case PacketImpl.DISCONNECT_V3:
return version >= PacketImpl.ARTEMIS_2_18_0_VERSION;
default:
return true;
}
@ -689,6 +693,11 @@ public final class ChannelImpl implements Channel {
}
}
@Override
public boolean isLocked() {
return failingOver;
}
@Override
public void lock() {
if (logger.isTraceEnabled()) {

View File

@ -29,10 +29,12 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CheckFailo
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V3;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V4;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateAddressMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateQueueMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateQueueMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionResponseMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSharedQueueMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSharedQueueMessage_V2;
@ -40,6 +42,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.Disconnect
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectConsumerWithKillMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage_V3;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.FederationDownstreamConnectMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.NullResponseMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.NullResponseMessage_V2;
@ -98,8 +101,10 @@ import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CHE
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CLUSTER_TOPOLOGY;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CLUSTER_TOPOLOGY_V2;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CLUSTER_TOPOLOGY_V3;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CLUSTER_TOPOLOGY_V4;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CREATESESSION;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CREATESESSION_RESP;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CREATESESSION_V2;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CREATE_ADDRESS;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CREATE_QUEUE;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.CREATE_QUEUE_V2;
@ -109,6 +114,7 @@ import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.DEL
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.DISCONNECT;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.DISCONNECT_CONSUMER;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.DISCONNECT_V2;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.DISCONNECT_V3;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.EXCEPTION;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.FEDERATION_DOWNSTREAM_CONNECT;
import static org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl.NULL_RESPONSE;
@ -477,6 +483,18 @@ public abstract class PacketDecoder implements Serializable {
packet = new FederationDownstreamConnectMessage();
break;
}
case CLUSTER_TOPOLOGY_V4: {
packet = new ClusterTopologyChangeMessage_V4();
break;
}
case CREATESESSION_V2: {
packet = new CreateSessionMessage_V2();
break;
}
case DISCONNECT_V3: {
packet = new DisconnectMessage_V3();
break;
}
default: {
throw ActiveMQClientMessageBundle.BUNDLE.invalidType(packetType);
}

View File

@ -38,6 +38,8 @@ public class PacketImpl implements Packet {
public static final int CONSUMER_PRIORITY_CHANGE_VERSION = ARTEMIS_2_7_0_VERSION;
public static final int FQQN_CHANGE_VERSION = ARTEMIS_2_7_0_VERSION;
// 2.18.0
public static final int ARTEMIS_2_18_0_VERSION = 131;
public static final SimpleString OLD_QUEUE_PREFIX = new SimpleString("jms.queue.");
public static final SimpleString OLD_TEMP_QUEUE_PREFIX = new SimpleString("jms.tempqueue.");
@ -279,6 +281,11 @@ public class PacketImpl implements Packet {
public static final byte FEDERATION_DOWNSTREAM_CONNECT = -16;
public static final byte CLUSTER_TOPOLOGY_V4 = -17;
public static final byte CREATESESSION_V2 = -18;
public static final byte DISCONNECT_V3 = -19;
// Static --------------------------------------------------------

View File

@ -26,9 +26,12 @@ import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.ActiveMQRedirectedException;
import org.apache.activemq.artemis.api.core.ActiveMQRemoteDisconnectException;
import org.apache.activemq.artemis.api.core.DisconnectReason;
import org.apache.activemq.artemis.api.core.Interceptor;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
import org.apache.activemq.artemis.core.client.ActiveMQClientLogger;
import org.apache.activemq.artemis.core.protocol.core.Channel;
@ -38,6 +41,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.ChannelImpl.CHANNEL_I
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectConsumerWithKillMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.DisconnectMessage_V3;
import org.apache.activemq.artemis.core.security.ActiveMQPrincipal;
import org.apache.activemq.artemis.spi.core.protocol.AbstractRemotingConnection;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
@ -206,7 +210,7 @@ public class RemotingConnectionImpl extends AbstractRemotingConnection implement
destroyed = true;
}
if (!(me instanceof ActiveMQRemoteDisconnectException)) {
if (!(me instanceof ActiveMQRemoteDisconnectException) && !(me instanceof ActiveMQRedirectedException)) {
ActiveMQClientLogger.LOGGER.connectionFailureDetected(transportConnection.getRemoteAddress(), me.getMessage(), me.getType());
}
@ -250,11 +254,16 @@ public class RemotingConnectionImpl extends AbstractRemotingConnection implement
@Override
public void disconnect(final boolean criticalError) {
disconnect(null, criticalError);
disconnect(criticalError ? DisconnectReason.SHOUT_DOWN_ON_CRITICAL_ERROR : DisconnectReason.SHOUT_DOWN, null, null);
}
@Override
public void disconnect(String scaleDownNodeID, final boolean criticalError) {
disconnect(criticalError ? DisconnectReason.SCALE_DOWN_ON_CRITICAL_ERROR : DisconnectReason.SCALE_DOWN, scaleDownNodeID, null);
}
@Override
public void disconnect(DisconnectReason reason, String targetNodeID, TransportConfiguration targetConnector) {
Channel channel0 = getChannel(ChannelImpl.CHANNEL_ID.PING.id, -1);
// And we remove all channels from the connection, this ensures no more packets will be processed after this
@ -263,7 +272,7 @@ public class RemotingConnectionImpl extends AbstractRemotingConnection implement
Set<Channel> allChannels = new HashSet<>(channels.values());
if (!criticalError) {
if (!reason.isCriticalError()) {
removeAllChannels();
} else {
// We can't hold a lock if a critical error is happening...
@ -273,15 +282,17 @@ public class RemotingConnectionImpl extends AbstractRemotingConnection implement
// Now we are 100% sure that no more packets will be processed we can flush then send the disconnect
if (!criticalError) {
if (!reason.isCriticalError()) {
for (Channel channel : allChannels) {
channel.flushConfirmations();
}
}
Packet disconnect;
if (channel0.supports(PacketImpl.DISCONNECT_V2)) {
disconnect = new DisconnectMessage_V2(nodeID, scaleDownNodeID);
if (channel0.supports(PacketImpl.DISCONNECT_V3)) {
disconnect = new DisconnectMessage_V3(nodeID, reason, SimpleString.toSimpleString(targetNodeID), targetConnector);
} else if (channel0.supports(PacketImpl.DISCONNECT_V2)) {
disconnect = new DisconnectMessage_V2(nodeID, reason.isScaleDown() ? targetNodeID : null);
} else {
disconnect = new DisconnectMessage(nodeID);
}

View File

@ -30,7 +30,17 @@ public class ClusterTopologyChangeMessage_V3 extends ClusterTopologyChangeMessag
final String scaleDownGroupName,
final Pair<TransportConfiguration, TransportConfiguration> pair,
final boolean last) {
super(CLUSTER_TOPOLOGY_V3);
this(CLUSTER_TOPOLOGY_V3, uniqueEventID, nodeID, backupGroupName, scaleDownGroupName, pair, last);
}
protected ClusterTopologyChangeMessage_V3(final byte type,
final long uniqueEventID,
final String nodeID,
final String backupGroupName,
final String scaleDownGroupName,
final Pair<TransportConfiguration, TransportConfiguration> pair,
final boolean last) {
super(type);
this.nodeID = nodeID;
@ -51,6 +61,10 @@ public class ClusterTopologyChangeMessage_V3 extends ClusterTopologyChangeMessag
super(CLUSTER_TOPOLOGY_V3);
}
public ClusterTopologyChangeMessage_V3(byte type) {
super(type);
}
public String getScaleDownGroupName() {
return scaleDownGroupName;
}
@ -75,8 +89,17 @@ public class ClusterTopologyChangeMessage_V3 extends ClusterTopologyChangeMessag
return result;
}
@Override
protected String getParentString() {
return toString(false);
}
@Override
public String toString() {
return toString(true);
}
private String toString(boolean closed) {
StringBuffer buff = new StringBuffer(getParentString());
buff.append(", exit=" + exit);
buff.append(", last=" + last);
@ -85,7 +108,9 @@ public class ClusterTopologyChangeMessage_V3 extends ClusterTopologyChangeMessag
buff.append(", backupGroupName=" + backupGroupName);
buff.append(", uniqueEventID=" + uniqueEventID);
buff.append(", scaleDownGroupName=" + scaleDownGroupName);
buff.append("]");
if (closed) {
buff.append("]");
}
return buff.toString();
}

View File

@ -0,0 +1,91 @@
/*
* 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
*
* 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.apache.activemq.artemis.core.protocol.core.impl.wireformat;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.Pair;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
public class ClusterTopologyChangeMessage_V4 extends ClusterTopologyChangeMessage_V3 {
private int serverVersion;
public ClusterTopologyChangeMessage_V4(final long uniqueEventID,
final String nodeID,
final String backupGroupName,
final String scaleDownGroupName,
final Pair<TransportConfiguration, TransportConfiguration> pair,
final boolean last,
final int serverVersion) {
super(CLUSTER_TOPOLOGY_V4, uniqueEventID, nodeID, backupGroupName, scaleDownGroupName, pair, last);
this.serverVersion = serverVersion;
}
public ClusterTopologyChangeMessage_V4() {
super(CLUSTER_TOPOLOGY_V4);
}
public int getServerVersion() {
return serverVersion;
}
@Override
public void encodeRest(final ActiveMQBuffer buffer) {
super.encodeRest(buffer);
buffer.writeInt(serverVersion);
}
@Override
public void decodeRest(final ActiveMQBuffer buffer) {
super.decodeRest(buffer);
serverVersion = buffer.readInt();
}
@Override
public int hashCode() {
final int prime = 31;
int result = super.hashCode();
result = prime * result + serverVersion;
return result;
}
@Override
public String toString() {
StringBuffer buf = new StringBuffer(getParentString());
buf.append(", clientVersion=" + serverVersion);
buf.append("]");
return buf.toString();
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (!super.equals(obj)) {
return false;
}
if (!(obj instanceof ClusterTopologyChangeMessage_V4)) {
return false;
}
ClusterTopologyChangeMessage_V4 other = (ClusterTopologyChangeMessage_V4) obj;
return serverVersion == other.serverVersion;
}
}

View File

@ -57,7 +57,23 @@ public class CreateSessionMessage extends PacketImpl {
final boolean preAcknowledge,
final int windowSize,
final String defaultAddress) {
super(CREATESESSION);
this(CREATESESSION, name, sessionChannelID, version, username, password, minLargeMessageSize, xa, autoCommitSends, autoCommitAcks, preAcknowledge, windowSize, defaultAddress);
}
protected CreateSessionMessage(final byte type,
final String name,
final long sessionChannelID,
final int version,
final String username,
final String password,
final int minLargeMessageSize,
final boolean xa,
final boolean autoCommitSends,
final boolean autoCommitAcks,
final boolean preAcknowledge,
final int windowSize,
final String defaultAddress) {
super(type);
this.name = name;
@ -88,6 +104,10 @@ public class CreateSessionMessage extends PacketImpl {
super(CREATESESSION);
}
protected CreateSessionMessage(final byte type) {
super(type);
}
// Public --------------------------------------------------------
public String getName() {
@ -194,9 +214,18 @@ public class CreateSessionMessage extends PacketImpl {
return result;
}
@Override
protected String getParentString() {
return toString(false);
}
@Override
public String toString() {
StringBuffer buff = new StringBuffer(getParentString());
return toString(true);
}
private String toString(boolean closed) {
StringBuffer buff = new StringBuffer(super.getParentString());
buff.append(", autoCommitAcks=" + autoCommitAcks);
buff.append(", autoCommitSends=" + autoCommitSends);
buff.append(", defaultAddress=" + defaultAddress);
@ -209,7 +238,9 @@ public class CreateSessionMessage extends PacketImpl {
buff.append(", version=" + version);
buff.append(", windowSize=" + windowSize);
buff.append(", xa=" + xa);
buff.append("]");
if (closed) {
buff.append("]");
}
return buff.toString();
}

View File

@ -0,0 +1,103 @@
/*
* 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
*
* 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.apache.activemq.artemis.core.protocol.core.impl.wireformat;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
public class CreateSessionMessage_V2 extends CreateSessionMessage {
private String clientID = null;
public CreateSessionMessage_V2(final String name,
final long sessionChannelID,
final int version,
final String username,
final String password,
final int minLargeMessageSize,
final boolean xa,
final boolean autoCommitSends,
final boolean autoCommitAcks,
final boolean preAcknowledge,
final int windowSize,
final String defaultAddress,
final String clientID) {
super(CREATESESSION_V2, name, sessionChannelID, version, username, password, minLargeMessageSize, xa, autoCommitSends, autoCommitAcks, preAcknowledge, windowSize, defaultAddress);
this.clientID = clientID;
}
public CreateSessionMessage_V2() {
super(CREATESESSION_V2);
}
// Public --------------------------------------------------------
public String getClientID() {
return clientID;
}
@Override
public void encodeRest(final ActiveMQBuffer buffer) {
super.encodeRest(buffer);
buffer.writeNullableString(clientID);
}
@Override
public void decodeRest(final ActiveMQBuffer buffer) {
super.decodeRest(buffer);
clientID = buffer.readNullableString();
}
@Override
public String toString() {
StringBuffer buf = new StringBuffer(getParentString());
buf.append(", metadata=" + clientID);
buf.append("]");
return buf.toString();
}
@Override
public int hashCode() {
final int prime = 31;
int result = super.hashCode();
result = prime * result + ((clientID == null) ? 0 : clientID.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (!super.equals(obj)) {
return false;
}
if (!(obj instanceof CreateSessionMessage_V2)) {
return false;
}
CreateSessionMessage_V2 other = (CreateSessionMessage_V2) obj;
if (clientID == null) {
if (other.clientID != null)
return false;
} else if (!clientID.equals(other.clientID))
return false;
return true;
}
}

View File

@ -0,0 +1,137 @@
/*
* 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
*
* 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.apache.activemq.artemis.core.protocol.core.impl.wireformat;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.DisconnectReason;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
public class DisconnectMessage_V3 extends DisconnectMessage {
private DisconnectReason reason;
private SimpleString targetNodeID;
private TransportConfiguration targetConnector;
public DisconnectMessage_V3(final SimpleString nodeID,
final DisconnectReason reason,
final SimpleString targetNodeID,
final TransportConfiguration targetConnector) {
super(DISCONNECT_V3);
this.nodeID = nodeID;
this.reason = reason;
this.targetNodeID = targetNodeID;
this.targetConnector = targetConnector;
}
public DisconnectMessage_V3() {
super(DISCONNECT_V3);
}
// Public --------------------------------------------------------
public DisconnectReason getReason() {
return reason;
}
public SimpleString getTargetNodeID() {
return targetNodeID;
}
public TransportConfiguration getTargetConnector() {
return targetConnector;
}
@Override
public void encodeRest(final ActiveMQBuffer buffer) {
super.encodeRest(buffer);
buffer.writeByte(reason == null ? -1 : reason.getType());
buffer.writeNullableSimpleString(targetNodeID);
if (targetConnector != null) {
buffer.writeBoolean(true);
targetConnector.encode(buffer);
} else {
buffer.writeBoolean(false);
}
}
@Override
public void decodeRest(final ActiveMQBuffer buffer) {
super.decodeRest(buffer);
reason = DisconnectReason.getType(buffer.readByte());
targetNodeID = buffer.readNullableSimpleString();
boolean hasTargetConnector = buffer.readBoolean();
if (hasTargetConnector) {
targetConnector = new TransportConfiguration();
targetConnector.decode(buffer);
} else {
targetConnector = null;
}
}
@Override
public String toString() {
StringBuffer buf = new StringBuffer(getParentString());
buf.append(", nodeID=" + nodeID);
buf.append(", reason=" + reason);
buf.append(", targetNodeID=" + targetNodeID);
buf.append(", targetConnector=" + targetConnector);
buf.append("]");
return buf.toString();
}
@Override
public int hashCode() {
final int prime = 31;
int result = super.hashCode();
result = prime * result + (reason.getType());
result = prime * result + ((targetNodeID == null) ? 0 : targetNodeID.hashCode());
result = prime * result + ((targetConnector == null) ? 0 : targetConnector.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (!super.equals(obj)) {
return false;
}
if (!(obj instanceof DisconnectMessage_V3)) {
return false;
}
DisconnectMessage_V3 other = (DisconnectMessage_V3) obj;
if (reason == null) {
if (other.reason != null)
return false;
} else if (!reason.equals(other.reason))
return false;
if (targetNodeID == null) {
if (other.targetNodeID != null) {
return false;
}
} else if (!targetNodeID.equals(other.targetNodeID)) {
return false;
}
return true;
}
}

View File

@ -361,6 +361,10 @@ public class TransportConstants {
public static final boolean DEFAULT_PROXY_REMOTE_DNS = false;
public static final String REDIRECT_TO = "redirect-to";
public static final String DEFAULT_REDIRECT_TO = null;
private static int parseDefaultVariable(String variableName, int defaultValue) {
try {
String variable = System.getProperty(TransportConstants.class.getName() + "." + variableName);
@ -437,6 +441,7 @@ public class TransportConstants {
allowableAcceptorKeys.add(TransportConstants.QUIET_PERIOD);
allowableAcceptorKeys.add(TransportConstants.DISABLE_STOMP_SERVER_HEADER);
allowableAcceptorKeys.add(TransportConstants.AUTO_START);
allowableAcceptorKeys.add(TransportConstants.REDIRECT_TO);
ALLOWABLE_ACCEPTOR_KEYS = Collections.unmodifiableSet(allowableAcceptorKeys);

View File

@ -21,6 +21,7 @@ import java.util.concurrent.Future;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.DisconnectReason;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.core.remoting.CloseListener;
@ -183,6 +184,13 @@ public interface RemotingConnection extends BufferHandler {
*/
void disconnect(String scaleDownNodeID, boolean criticalError);
/**
* Disconnect the connection, closing all channels
*/
default void disconnect(DisconnectReason reason, String targetNodeID, TransportConfiguration targetConnector) {
disconnect(reason.isScaleDown() ? targetNodeID : null, reason.isCriticalError());
}
/**
* returns true if any data has been received since the last time this method was called.
*

View File

@ -69,7 +69,8 @@ public interface ClientProtocolManager {
boolean autoCommitAcks,
boolean preAcknowledge,
int minLargeMessageSize,
int confirmationWindowSize) throws ActiveMQException;
int confirmationWindowSize,
String clientID) throws ActiveMQException;
boolean cleanupBeforeFailover(ActiveMQException cause);

View File

@ -172,4 +172,12 @@ public interface Connection {
//returns true if one of the configs points to the same
//node as this connection does.
boolean isSameTarget(TransportConfiguration... configs);
default String getSNIHostName() {
return null;
}
default String getRedirectTo() {
return null;
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.activemq.artemis.spi.core.remoting;
import org.apache.activemq.artemis.api.core.DisconnectReason;
import org.apache.activemq.artemis.api.core.Pair;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
@ -23,7 +24,7 @@ import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
public interface TopologyResponseHandler {
// This is sent when the server is telling the client the node is being disconnected
void nodeDisconnected(RemotingConnection conn, String nodeID, String scaleDownTargetNodeID);
void nodeDisconnected(RemotingConnection conn, String nodeID, DisconnectReason reason, String targetNodeID, TransportConfiguration tagetConnector);
void notifyNodeUp(long uniqueEventID,
String backupGroupName,

View File

@ -20,4 +20,4 @@ activemq.version.minorVersion=${activemq.version.minorVersion}
activemq.version.microVersion=${activemq.version.microVersion}
activemq.version.incrementingVersion=${activemq.version.incrementingVersion}
activemq.version.versionTag=${activemq.version.versionTag}
activemq.version.compatibleVersionList=121,122,123,124,125,126,127,128,129,130
activemq.version.compatibleVersionList=121,122,123,124,125,126,127,128,129,130,131

View File

@ -42,7 +42,8 @@ public class OperationAnnotationTest {
{DivertControl.class},
{AcceptorControl.class},
{ClusterConnectionControl.class},
{BroadcastGroupControl.class}});
{BroadcastGroupControl.class},
{BrokerBalancerControl.class}});
}
private Class<?> managementClass;

View File

@ -79,14 +79,6 @@ public class ActiveMQConnection extends ActiveMQConnectionForContextImpl impleme
public static final SimpleString CONNECTION_ID_PROPERTY_NAME = MessageUtil.CONNECTION_ID_PROPERTY_NAME;
/**
* Just like {@link ClientSession.AddressQuery#JMS_SESSION_IDENTIFIER_PROPERTY} this is
* used to identify the ClientID over JMS Session.
* However this is only used when the JMS Session.clientID is set (which is optional).
* With this property management tools and the server can identify the jms-client-id used over JMS
*/
public static String JMS_SESSION_CLIENT_ID_PROPERTY = "jms-client-id";
// Static ---------------------------------------------------------------------------------------
// Attributes -----------------------------------------------------------------------------------
@ -271,7 +263,7 @@ public class ActiveMQConnection extends ActiveMQConnectionForContextImpl impleme
private void validateClientID(ClientSession validateSession, String clientID)
throws InvalidClientIDException, ActiveMQException {
try {
validateSession.addUniqueMetaData(JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
validateSession.addUniqueMetaData(ClientSession.JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
} catch (ActiveMQException e) {
if (e.getType() == ActiveMQExceptionType.DUPLICATE_METADATA) {
throw new InvalidClientIDException("clientID=" + clientID + " was already set into another connection");
@ -605,17 +597,17 @@ public class ActiveMQConnection extends ActiveMQConnectionForContextImpl impleme
boolean isBlockOnAcknowledge = sessionFactory.getServerLocator().isBlockOnAcknowledge();
int ackBatchSize = sessionFactory.getServerLocator().getAckBatchSize();
if (acknowledgeMode == Session.SESSION_TRANSACTED) {
session = sessionFactory.createSession(username, password, isXA, false, false, sessionFactory.getServerLocator().isPreAcknowledge(), transactionBatchSize);
session = sessionFactory.createSession(username, password, isXA, false, false, sessionFactory.getServerLocator().isPreAcknowledge(), transactionBatchSize, clientID);
} else if (acknowledgeMode == Session.AUTO_ACKNOWLEDGE) {
session = sessionFactory.createSession(username, password, isXA, true, true, sessionFactory.getServerLocator().isPreAcknowledge(), 0);
session = sessionFactory.createSession(username, password, isXA, true, true, sessionFactory.getServerLocator().isPreAcknowledge(), 0, clientID);
} else if (acknowledgeMode == Session.DUPS_OK_ACKNOWLEDGE) {
session = sessionFactory.createSession(username, password, isXA, true, true, sessionFactory.getServerLocator().isPreAcknowledge(), dupsOKBatchSize);
session = sessionFactory.createSession(username, password, isXA, true, true, sessionFactory.getServerLocator().isPreAcknowledge(), dupsOKBatchSize, clientID);
} else if (acknowledgeMode == Session.CLIENT_ACKNOWLEDGE) {
session = sessionFactory.createSession(username, password, isXA, true, false, sessionFactory.getServerLocator().isPreAcknowledge(), isBlockOnAcknowledge ? transactionBatchSize : ackBatchSize);
session = sessionFactory.createSession(username, password, isXA, true, false, sessionFactory.getServerLocator().isPreAcknowledge(), isBlockOnAcknowledge ? transactionBatchSize : ackBatchSize, clientID);
} else if (acknowledgeMode == ActiveMQJMSConstants.INDIVIDUAL_ACKNOWLEDGE) {
session = sessionFactory.createSession(username, password, isXA, true, false, false, isBlockOnAcknowledge ? transactionBatchSize : ackBatchSize);
session = sessionFactory.createSession(username, password, isXA, true, false, false, isBlockOnAcknowledge ? transactionBatchSize : ackBatchSize, clientID);
} else if (acknowledgeMode == ActiveMQJMSConstants.PRE_ACKNOWLEDGE) {
session = sessionFactory.createSession(username, password, isXA, true, false, true, transactionBatchSize);
session = sessionFactory.createSession(username, password, isXA, true, false, true, transactionBatchSize, clientID);
} else {
throw new JMSRuntimeException("Invalid ackmode: " + acknowledgeMode);
}
@ -636,8 +628,6 @@ public class ActiveMQConnection extends ActiveMQConnectionForContextImpl impleme
session.start();
}
this.addSessionMetaData(session);
return jbs;
} catch (ActiveMQException e) {
throw JMSExceptionHelper.convertFromActiveMQException(e);
@ -681,13 +671,13 @@ public class ActiveMQConnection extends ActiveMQConnectionForContextImpl impleme
public void authorize(boolean validateClientId) throws JMSException {
try {
initialSession = sessionFactory.createSession(username, password, false, false, false, false, 0);
initialSession = sessionFactory.createSession(username, password, false, false, false, false, 0, clientID);
if (clientID != null) {
if (validateClientId) {
validateClientID(initialSession, clientID);
} else {
initialSession.addMetaData(JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
initialSession.addMetaData(ClientSession.JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
}
}
@ -703,7 +693,7 @@ public class ActiveMQConnection extends ActiveMQConnectionForContextImpl impleme
private void addSessionMetaData(ClientSession session) throws ActiveMQException {
session.addMetaData(ClientSession.JMS_SESSION_IDENTIFIER_PROPERTY, "");
if (clientID != null) {
session.addMetaData(JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
session.addMetaData(ClientSession.JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
}
}

View File

@ -36,6 +36,7 @@ import org.apache.activemq.artemis.protocol.amqp.client.ProtonClientProtocolMana
import org.apache.activemq.artemis.protocol.amqp.connect.mirror.ReferenceNodeStore;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPConnectionContext;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPConstants;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPRedirectHandler;
import org.apache.activemq.artemis.protocol.amqp.proton.AmqpSupport;
import org.apache.activemq.artemis.protocol.amqp.sasl.ClientSASLFactory;
import org.apache.activemq.artemis.protocol.amqp.sasl.MechanismFinder;
@ -53,7 +54,7 @@ import org.jboss.logging.Logger;
/**
* A proton protocol manager, basically reads the Proton Input and maps proton resources to ActiveMQ Artemis resources
*/
public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage, AmqpInterceptor, ActiveMQProtonRemotingConnection> implements NotificationListener {
public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage, AmqpInterceptor, ActiveMQProtonRemotingConnection, AMQPRedirectHandler> implements NotificationListener {
private static final Logger logger = Logger.getLogger(ProtonProtocolManager.class);
@ -105,6 +106,7 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,
private boolean directDeliver = true;
private final AMQPRedirectHandler redirectHandler;
/*
* used when you want to treat senders as a subscription on an address rather than consuming from the actual queue for
@ -118,6 +120,7 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,
this.factory = factory;
this.server = server;
this.updateInterceptors(incomingInterceptors, outgoingInterceptors);
redirectHandler = new AMQPRedirectHandler(server);
}
public synchronized ReferenceNodeStore getReferenceIDSupplier() {
@ -349,6 +352,11 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,
return prefixes;
}
@Override
public AMQPRedirectHandler getRedirectHandler() {
return redirectHandler;
}
public String invokeIncoming(AMQPMessage message, ActiveMQProtonRemotingConnection connection) {
return super.invokeInterceptors(this.incomingInterceptors, message, connection);
}

View File

@ -95,7 +95,7 @@ public class ProtonClientProtocolManager extends ProtonProtocolManager implement
}
@Override
public SessionContext createSessionContext(String name, String username, String password, boolean xa, boolean autoCommitSends, boolean autoCommitAcks, boolean preAcknowledge, int minLargeMessageSize, int confirmationWindowSize) throws ActiveMQException {
public SessionContext createSessionContext(String name, String username, String password, boolean xa, boolean autoCommitSends, boolean autoCommitAcks, boolean preAcknowledge, int minLargeMessageSize, int confirmationWindowSize, String clientID) throws ActiveMQException {
throw new UnsupportedOperationException();
}

View File

@ -215,7 +215,8 @@ public class AMQPBrokerConnectionManager implements ActiveMQComponent, ClientCon
boolean autoCommitAcks,
boolean preAcknowledge,
int minLargeMessageSize,
int confirmationWindowSize) throws ActiveMQException {
int confirmationWindowSize,
String clientID) throws ActiveMQException {
return null;
}

View File

@ -525,7 +525,9 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
} catch (Exception e) {
log.error("Error init connection", e);
}
if (!validateConnection(connection)) {
if ((connectionCallback.getTransportConnection().getRedirectTo() != null && protocolManager.getRedirectHandler()
.redirect(this, connection)) || !validateConnection(connection)) {
connection.close();
} else {
connection.setContext(AMQPConnectionContext.this);

View File

@ -0,0 +1,37 @@
/**
* 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.protocol.amqp.proton;
import org.apache.activemq.artemis.core.server.balancing.RedirectContext;
import org.apache.qpid.proton.engine.Connection;
public class AMQPRedirectContext extends RedirectContext {
private final Connection protonConnection;
public Connection getProtonConnection() {
return protonConnection;
}
public AMQPRedirectContext(AMQPConnectionContext connectionContext, Connection protonConnection) {
super(connectionContext.getConnectionCallback().getProtonConnectionDelegate(), connectionContext.getRemoteContainer(),
connectionContext.getSASLResult() != null ? connectionContext.getSASLResult().getUser() : null);
this.protonConnection = protonConnection;
}
}

View File

@ -0,0 +1,64 @@
/**
* 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.protocol.amqp.proton;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.balancing.RedirectHandler;
import org.apache.activemq.artemis.utils.ConfigurationHelper;
import org.apache.qpid.proton.amqp.transport.ConnectionError;
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
import org.apache.qpid.proton.engine.Connection;
import java.util.HashMap;
import java.util.Map;
public class AMQPRedirectHandler extends RedirectHandler<AMQPRedirectContext> {
public AMQPRedirectHandler(ActiveMQServer server) {
super(server);
}
public boolean redirect(AMQPConnectionContext connectionContext, Connection protonConnection) throws Exception {
return redirect(new AMQPRedirectContext(connectionContext, protonConnection));
}
@Override
protected void cannotRedirect(AMQPRedirectContext context) throws Exception {
ErrorCondition error = new ErrorCondition();
error.setCondition(ConnectionError.CONNECTION_FORCED);
error.setDescription(String.format("Broker balancer %s is not ready to redirect", context.getConnection().getTransportConnection().getRedirectTo()));
context.getProtonConnection().setCondition(error);
}
@Override
protected void redirectTo(AMQPRedirectContext context) throws Exception {
String host = ConfigurationHelper.getStringProperty(TransportConstants.HOST_PROP_NAME, TransportConstants.DEFAULT_HOST, context.getTarget().getConnector().getParams());
int port = ConfigurationHelper.getIntProperty(TransportConstants.PORT_PROP_NAME, TransportConstants.DEFAULT_PORT, context.getTarget().getConnector().getParams());
ErrorCondition error = new ErrorCondition();
error.setCondition(ConnectionError.REDIRECT);
error.setDescription(String.format("Connection redirected to %s:%d by broker balancer %s", host, port, context.getConnection().getTransportConnection().getRedirectTo()));
Map info = new HashMap();
info.put(AmqpSupport.NETWORK_HOST, host);
info.put(AmqpSupport.PORT, port);
error.setInfo(info);
context.getProtonConnection().setCondition(error);
}
}

View File

@ -18,6 +18,7 @@ package org.apache.activemq.artemis.protocol.amqp.sasl;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.protocol.amqp.broker.AmqpInterceptor;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPRedirectHandler;
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
@ -30,7 +31,7 @@ public class AnonymousServerSASLFactory implements ServerSASLFactory {
}
@Override
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor> manager, Connection connection,
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor, AMQPRedirectHandler> manager, Connection connection,
RemotingConnection remotingConnection) {
return new AnonymousServerSASL();
}

View File

@ -21,6 +21,7 @@ import java.security.Principal;
import org.apache.activemq.artemis.core.remoting.CertificateUtil;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.protocol.amqp.broker.AmqpInterceptor;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPRedirectHandler;
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
@ -39,7 +40,7 @@ public class ExternalServerSASLFactory implements ServerSASLFactory {
}
@Override
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor> manager, Connection connection,
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor, AMQPRedirectHandler> manager, Connection connection,
RemotingConnection remotingConnection) {
// validate ssl cert present
Principal principal = CertificateUtil.getPeerPrincipalFromConnection(remotingConnection);

View File

@ -19,6 +19,7 @@ package org.apache.activemq.artemis.protocol.amqp.sasl;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.protocol.amqp.broker.AmqpInterceptor;
import org.apache.activemq.artemis.protocol.amqp.broker.ProtonProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPRedirectHandler;
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
@ -34,7 +35,7 @@ public class GSSAPIServerSASLFactory implements ServerSASLFactory {
}
@Override
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor> manager, Connection connection,
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor, AMQPRedirectHandler> manager, Connection connection,
RemotingConnection remotingConnection) {
if (manager instanceof ProtonProtocolManager) {
GSSAPIServerSASL gssapiServerSASL = new GSSAPIServerSASL();

View File

@ -18,6 +18,7 @@ package org.apache.activemq.artemis.protocol.amqp.sasl;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.protocol.amqp.broker.AmqpInterceptor;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPRedirectHandler;
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
@ -30,7 +31,7 @@ public class PlainServerSASLFactory implements ServerSASLFactory {
}
@Override
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor> manager, Connection connection,
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor, AMQPRedirectHandler> manager, Connection connection,
RemotingConnection remotingConnection) {
return new PlainSASL(server.getSecurityStore(), manager.getSecurityDomain(), connection.getProtocolConnection());
}

View File

@ -18,6 +18,7 @@ package org.apache.activemq.artemis.protocol.amqp.sasl;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.protocol.amqp.broker.AmqpInterceptor;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPRedirectHandler;
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
@ -40,7 +41,7 @@ public interface ServerSASLFactory {
* @param remotingConnection
* @return a new instance of {@link ServerSASL} that implements the provided mechanism
*/
ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor> manager, Connection connection,
ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor, AMQPRedirectHandler> manager, Connection connection,
RemotingConnection remotingConnection);
/**

View File

@ -31,6 +31,7 @@ import javax.security.auth.login.LoginException;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.protocol.amqp.broker.AmqpInterceptor;
import org.apache.activemq.artemis.protocol.amqp.broker.ProtonProtocolManager;
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPRedirectHandler;
import org.apache.activemq.artemis.protocol.amqp.sasl.ServerSASL;
import org.apache.activemq.artemis.protocol.amqp.sasl.ServerSASLFactory;
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
@ -67,7 +68,7 @@ public abstract class SCRAMServerSASLFactory implements ServerSASLFactory {
}
@Override
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor> manager, Connection connection,
public ServerSASL create(ActiveMQServer server, ProtocolManager<AmqpInterceptor, AMQPRedirectHandler> manager, Connection connection,
RemotingConnection remotingConnection) {
try {
if (manager instanceof ProtonProtocolManager) {

View File

@ -23,11 +23,9 @@ import org.apache.activemq.artemis.core.protocol.core.Channel;
import org.apache.activemq.artemis.core.protocol.core.Packet;
import org.apache.activemq.artemis.core.protocol.core.impl.ActiveMQClientProtocolManager;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionResponseMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SubscribeClusterTopologyUpdatesMessageV2;
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnectorFactory;
import org.apache.activemq.artemis.core.version.Version;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
import org.apache.activemq.artemis.spi.core.remoting.SessionContext;
@ -49,7 +47,7 @@ public class HornetQClientProtocolManager extends ActiveMQClientProtocolManager
}
@Override
protected Packet newCreateSessionPacket(Version clientVersion,
protected Packet newCreateSessionPacket(int clientVersion,
String name,
String username,
String password,
@ -59,8 +57,9 @@ public class HornetQClientProtocolManager extends ActiveMQClientProtocolManager
boolean preAcknowledge,
int minLargeMessageSize,
int confirmationWindowSize,
long sessionChannelID) {
return new CreateSessionMessage(name, sessionChannelID, VERSION_PLAYED, username, password, minLargeMessageSize, xa, autoCommitSends, autoCommitAcks, preAcknowledge, confirmationWindowSize, null);
long sessionChannelID,
String clientID) {
return super.newCreateSessionPacket(VERSION_PLAYED, name, username, password, xa, autoCommitSends, autoCommitAcks, preAcknowledge, minLargeMessageSize, confirmationWindowSize, sessionChannelID, clientID);
}
@Override

View File

@ -28,6 +28,7 @@ import io.netty.handler.codec.mqtt.MqttFixedHeader;
import io.netty.handler.codec.mqtt.MqttMessage;
import io.netty.handler.codec.mqtt.MqttMessageIdVariableHeader;
import io.netty.handler.codec.mqtt.MqttMessageType;
import io.netty.handler.codec.mqtt.MqttProperties;
import io.netty.handler.codec.mqtt.MqttPubAckMessage;
import io.netty.handler.codec.mqtt.MqttPublishMessage;
import io.netty.handler.codec.mqtt.MqttPublishVariableHeader;
@ -176,10 +177,13 @@ public class MQTTProtocolHandler extends ChannelInboundHandlerAdapter {
* @param connect
*/
void handleConnect(MqttConnectMessage connect) throws Exception {
connectionEntry.ttl = connect.variableHeader().keepAliveTimeSeconds() * 1500L;
if (connection.getTransportConnection().getRedirectTo() == null ||
!protocolManager.getRedirectHandler().redirect(connection, session, connect)) {
connectionEntry.ttl = connect.variableHeader().keepAliveTimeSeconds() * 1500L;
String clientId = connect.payload().clientIdentifier();
session.getConnectionManager().connect(clientId, connect.payload().userName(), connect.payload().passwordInBytes(), connect.variableHeader().isWillFlag(), connect.payload().willMessageInBytes(), connect.payload().willTopic(), connect.variableHeader().isWillRetain(), connect.variableHeader().willQos(), connect.variableHeader().isCleanSession());
String clientId = connect.payload().clientIdentifier();
session.getConnectionManager().connect(clientId, connect.payload().userName(), connect.payload().passwordInBytes(), connect.variableHeader().isWillFlag(), connect.payload().willMessageInBytes(), connect.payload().willTopic(), connect.variableHeader().isWillRetain(), connect.variableHeader().willQos(), connect.variableHeader().isCleanSession());
}
}
void disconnect(boolean error) {
@ -187,8 +191,12 @@ public class MQTTProtocolHandler extends ChannelInboundHandlerAdapter {
}
void sendConnack(MqttConnectReturnCode returnCode) {
sendConnack(returnCode, MqttProperties.NO_PROPERTIES);
}
void sendConnack(MqttConnectReturnCode returnCode, MqttProperties properties) {
MqttFixedHeader fixedHeader = new MqttFixedHeader(MqttMessageType.CONNACK, false, MqttQoS.AT_MOST_ONCE, false, 0);
MqttConnAckVariableHeader varHeader = new MqttConnAckVariableHeader(returnCode, true);
MqttConnAckVariableHeader varHeader = new MqttConnAckVariableHeader(returnCode, true, properties);
MqttConnAckMessage message = new MqttConnAckMessage(fixedHeader, varHeader);
sendToClient(message);
}

View File

@ -48,7 +48,7 @@ import org.apache.activemq.artemis.utils.collections.TypedProperties;
/**
* MQTTProtocolManager
*/
public class MQTTProtocolManager extends AbstractProtocolManager<MqttMessage, MQTTInterceptor, MQTTConnection> implements NotificationListener {
public class MQTTProtocolManager extends AbstractProtocolManager<MqttMessage, MQTTInterceptor, MQTTConnection, MQTTRedirectHandler> implements NotificationListener {
private static final List<String> websocketRegistryNames = Arrays.asList("mqtt", "mqttv3.1");
@ -62,6 +62,8 @@ public class MQTTProtocolManager extends AbstractProtocolManager<MqttMessage, MQ
private final Map<String, MQTTConnection> connectedClients;
private final Map<String, MQTTSessionState> sessionStates;
private final MQTTRedirectHandler redirectHandler;
MQTTProtocolManager(ActiveMQServer server,
Map<String, MQTTConnection> connectedClients,
Map<String, MQTTSessionState> sessionStates,
@ -72,6 +74,7 @@ public class MQTTProtocolManager extends AbstractProtocolManager<MqttMessage, MQ
this.sessionStates = sessionStates;
this.updateInterceptors(incomingInterceptors, outgoingInterceptors);
server.getManagementService().addNotificationListener(this);
redirectHandler = new MQTTRedirectHandler(server);
}
@Override
@ -209,6 +212,11 @@ public class MQTTProtocolManager extends AbstractProtocolManager<MqttMessage, MQ
return websocketRegistryNames;
}
@Override
public MQTTRedirectHandler getRedirectHandler() {
return redirectHandler;
}
public String invokeIncoming(MqttMessage mqttMessage, MQTTConnection connection) {
return super.invokeInterceptors(this.incomingInterceptors, mqttMessage, connection);
}

View File

@ -0,0 +1,37 @@
/**
* 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.core.protocol.mqtt;
import io.netty.handler.codec.mqtt.MqttConnectMessage;
import org.apache.activemq.artemis.core.server.balancing.RedirectContext;
public class MQTTRedirectContext extends RedirectContext {
private final MQTTSession mqttSession;
public MQTTSession getMQTTSession() {
return mqttSession;
}
public MQTTRedirectContext(MQTTConnection mqttConnection, MQTTSession mqttSession, MqttConnectMessage connect) {
super(mqttConnection, connect.payload().clientIdentifier(), connect.payload().userName());
this.mqttSession = mqttSession;
}
}

View File

@ -0,0 +1,55 @@
/**
* 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.core.protocol.mqtt;
import io.netty.handler.codec.mqtt.MqttConnectMessage;
import io.netty.handler.codec.mqtt.MqttConnectReturnCode;
import io.netty.handler.codec.mqtt.MqttProperties;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.balancing.RedirectHandler;
import org.apache.activemq.artemis.utils.ConfigurationHelper;
public class MQTTRedirectHandler extends RedirectHandler<MQTTRedirectContext> {
protected MQTTRedirectHandler(ActiveMQServer server) {
super(server);
}
public boolean redirect(MQTTConnection mqttConnection, MQTTSession mqttSession, MqttConnectMessage connect) throws Exception {
return redirect(new MQTTRedirectContext(mqttConnection, mqttSession, connect));
}
@Override
protected void cannotRedirect(MQTTRedirectContext context) throws Exception {
context.getMQTTSession().getProtocolHandler().sendConnack(MqttConnectReturnCode.CONNECTION_REFUSED_SERVER_UNAVAILABLE);
context.getMQTTSession().getProtocolHandler().disconnect(true);
}
@Override
protected void redirectTo(MQTTRedirectContext context) throws Exception {
String host = ConfigurationHelper.getStringProperty(TransportConstants.HOST_PROP_NAME, TransportConstants.DEFAULT_HOST, context.getTarget().getConnector().getParams());
int port = ConfigurationHelper.getIntProperty(TransportConstants.PORT_PROP_NAME, TransportConstants.DEFAULT_PORT, context.getTarget().getConnector().getParams());
MqttProperties mqttProperties = new MqttProperties();
mqttProperties.add(new MqttProperties.StringProperty(MqttProperties.MqttPropertyType.SERVER_REFERENCE.value(), String.format("%s:%d", host, port)));
context.getMQTTSession().getProtocolHandler().sendConnack(MqttConnectReturnCode.CONNECTION_REFUSED_USE_ANOTHER_SERVER, mqttProperties);
context.getMQTTSession().getProtocolHandler().disconnect(true);
}
}

View File

@ -1146,6 +1146,12 @@ public class OpenWireConnection extends AbstractRemotingConnection implements Se
@Override
public Response processAddConnection(ConnectionInfo info) throws Exception {
try {
if (transportConnection.getRedirectTo() != null && protocolManager.getRedirectHandler()
.redirect(OpenWireConnection.this, info)) {
shutdown(true);
return null;
}
protocolManager.addConnection(OpenWireConnection.this, info);
} catch (Exception e) {
Response resp = new ExceptionResponse(e);

View File

@ -81,7 +81,7 @@ import org.apache.activemq.util.LongSequenceGenerator;
import static org.apache.activemq.artemis.core.protocol.openwire.util.OpenWireUtil.SELECTOR_AWARE_OPTION;
public class OpenWireProtocolManager extends AbstractProtocolManager<Command, OpenWireInterceptor, OpenWireConnection> implements ClusterTopologyListener {
public class OpenWireProtocolManager extends AbstractProtocolManager<Command, OpenWireInterceptor, OpenWireConnection, OpenWireRedirectHandler> implements ClusterTopologyListener {
private static final List<String> websocketRegistryNames = Collections.EMPTY_LIST;
@ -137,6 +137,7 @@ public class OpenWireProtocolManager extends AbstractProtocolManager<Command, O
private final List<OpenWireInterceptor> incomingInterceptors = new ArrayList<>();
private final List<OpenWireInterceptor> outgoingInterceptors = new ArrayList<>();
private final OpenWireRedirectHandler redirectHandler;
protected static class VirtualTopicConfig {
public int filterPathTerminus;
@ -187,6 +188,8 @@ public class OpenWireProtocolManager extends AbstractProtocolManager<Command, O
//make sure we don't cluster advisories
clusterManager.addProtocolIgnoredAddress(AdvisorySupport.ADVISORY_TOPIC_PREFIX);
redirectHandler = new OpenWireRedirectHandler(server, this);
}
@Override
@ -636,6 +639,11 @@ public class OpenWireProtocolManager extends AbstractProtocolManager<Command, O
this.securityDomain = securityDomain;
}
@Override
public OpenWireRedirectHandler getRedirectHandler() {
return redirectHandler;
}
@Override
public String getSecurityDomain() {
return securityDomain;

View File

@ -0,0 +1,37 @@
/**
* 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.core.protocol.openwire;
import org.apache.activemq.artemis.core.server.balancing.RedirectContext;
import org.apache.activemq.command.ConnectionInfo;
public class OpenWireRedirectContext extends RedirectContext {
private final OpenWireConnection openWireConnection;
public OpenWireConnection getOpenWireConnection() {
return openWireConnection;
}
public OpenWireRedirectContext(OpenWireConnection openWireConnection, ConnectionInfo connectionInfo) {
super(openWireConnection.getRemotingConnection(), connectionInfo.getClientId(), connectionInfo.getUserName());
this.openWireConnection = openWireConnection;
}
}

View File

@ -0,0 +1,58 @@
/**
* 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.core.protocol.openwire;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.balancing.RedirectHandler;
import org.apache.activemq.artemis.utils.ConfigurationHelper;
import org.apache.activemq.command.ConnectionControl;
import org.apache.activemq.command.ConnectionInfo;
public class OpenWireRedirectHandler extends RedirectHandler<OpenWireRedirectContext> {
private final OpenWireProtocolManager protocolManager;
protected OpenWireRedirectHandler(ActiveMQServer server, OpenWireProtocolManager protocolManager) {
super(server);
this.protocolManager = protocolManager;
}
public boolean redirect(OpenWireConnection openWireConnection, ConnectionInfo connectionInfo) throws Exception {
if (!connectionInfo.isFaultTolerant()) {
throw new java.lang.IllegalStateException("Client not fault tolerant");
}
return redirect(new OpenWireRedirectContext(openWireConnection, connectionInfo));
}
@Override
protected void cannotRedirect(OpenWireRedirectContext context) throws Exception {
}
@Override
protected void redirectTo(OpenWireRedirectContext context) throws Exception {
String host = ConfigurationHelper.getStringProperty(TransportConstants.HOST_PROP_NAME, TransportConstants.DEFAULT_HOST, context.getTarget().getConnector().getParams());
int port = ConfigurationHelper.getIntProperty(TransportConstants.PORT_PROP_NAME, TransportConstants.DEFAULT_PORT, context.getTarget().getConnector().getParams());
ConnectionControl command = protocolManager.newConnectionControl();
command.setConnectedBrokers(String.format("tcp://%s:%d", host, port));
command.setRebalanceConnection(true);
context.getOpenWireConnection().dispatchSync(command);
}
}

View File

@ -38,6 +38,7 @@ import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.core.server.ServerSession;
import org.apache.activemq.artemis.core.server.balancing.RedirectHandler;
import org.apache.activemq.artemis.logs.AuditLogger;
import org.apache.activemq.artemis.spi.core.protocol.AbstractProtocolManager;
import org.apache.activemq.artemis.spi.core.protocol.ConnectionEntry;
@ -52,7 +53,7 @@ import static org.apache.activemq.artemis.core.protocol.stomp.ActiveMQStompProto
/**
* StompProtocolManager
*/
public class StompProtocolManager extends AbstractProtocolManager<StompFrame, StompFrameInterceptor, StompConnection> {
public class StompProtocolManager extends AbstractProtocolManager<StompFrame, StompFrameInterceptor, StompConnection, RedirectHandler> {
private static final List<String> websocketRegistryNames = Arrays.asList("v10.stomp", "v11.stomp", "v12.stomp");
@ -190,6 +191,11 @@ public class StompProtocolManager extends AbstractProtocolManager<StompFrame, St
return websocketRegistryNames;
}
@Override
public RedirectHandler getRedirectHandler() {
return null;
}
// Public --------------------------------------------------------
public boolean send(final StompConnection connection, final StompFrame frame) {

View File

@ -53,7 +53,6 @@ import org.apache.activemq.artemis.api.core.client.TopologyMember;
import org.apache.activemq.artemis.api.jms.ActiveMQJMSClient;
import org.apache.activemq.artemis.core.client.impl.ClientSessionInternal;
import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
import org.apache.activemq.artemis.jms.client.ActiveMQConnection;
import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
import org.apache.activemq.artemis.ra.ActiveMQRABundle;
@ -494,7 +493,7 @@ public class ActiveMQActivation {
result.addMetaData(ClientSession.JMS_SESSION_IDENTIFIER_PROPERTY, "");
String clientID = ra.getClientID() == null ? spec.getClientID() : ra.getClientID();
if (clientID != null) {
result.addMetaData(ActiveMQConnection.JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
result.addMetaData(ClientSession.JMS_SESSION_CLIENT_ID_PROPERTY, clientID);
}
logger.debug("Using queue connection " + result);

View File

@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectConfiguration;
import org.apache.activemq.artemis.core.config.balancing.BrokerBalancerConfiguration;
import org.apache.activemq.artemis.core.server.metrics.ActiveMQMetricsPlugin;
import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerFederationPlugin;
import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerAddressPlugin;
@ -466,6 +467,18 @@ public interface Configuration {
Configuration addDivertConfiguration(DivertConfiguration config);
/**
* Returns the redirects configured for this server.
*/
List<BrokerBalancerConfiguration> getBalancerConfigurations();
/**
* Sets the redirects configured for this server.
*/
Configuration setBalancerConfigurations(List<BrokerBalancerConfiguration> configs);
Configuration addBalancerConfiguration(BrokerBalancerConfiguration config);
/**
* Returns the cluster connections configured for this server.
* <p>

View File

@ -0,0 +1,95 @@
/*
* 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
*
* 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.apache.activemq.artemis.core.config.balancing;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetKey;
import java.io.Serializable;
public class BrokerBalancerConfiguration implements Serializable {
private String name = null;
private TargetKey targetKey = TargetKey.SOURCE_IP;
private String targetKeyFilter = null;
private String localTargetFilter = null;
private int cacheTimeout = -1;
private PoolConfiguration poolConfiguration = null;
private PolicyConfiguration policyConfiguration = null;
public String getName() {
return name;
}
public BrokerBalancerConfiguration setName(String name) {
this.name = name;
return this;
}
public TargetKey getTargetKey() {
return targetKey;
}
public BrokerBalancerConfiguration setTargetKey(TargetKey targetKey) {
this.targetKey = targetKey;
return this;
}
public String getTargetKeyFilter() {
return targetKeyFilter;
}
public BrokerBalancerConfiguration setTargetKeyFilter(String targetKeyFilter) {
this.targetKeyFilter = targetKeyFilter;
return this;
}
public String getLocalTargetFilter() {
return localTargetFilter;
}
public BrokerBalancerConfiguration setLocalTargetFilter(String localTargetFilter) {
this.localTargetFilter = localTargetFilter;
return this;
}
public int getCacheTimeout() {
return cacheTimeout;
}
public BrokerBalancerConfiguration setCacheTimeout(int cacheTimeout) {
this.cacheTimeout = cacheTimeout;
return this;
}
public PolicyConfiguration getPolicyConfiguration() {
return policyConfiguration;
}
public BrokerBalancerConfiguration setPolicyConfiguration(PolicyConfiguration policyConfiguration) {
this.policyConfiguration = policyConfiguration;
return this;
}
public PoolConfiguration getPoolConfiguration() {
return poolConfiguration;
}
public BrokerBalancerConfiguration setPoolConfiguration(PoolConfiguration poolConfiguration) {
this.poolConfiguration = poolConfiguration;
return this;
}
}

View File

@ -0,0 +1,45 @@
/**
* 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.core.config.balancing;
import java.io.Serializable;
import java.util.Map;
public class PolicyConfiguration implements Serializable {
private String name;
private Map<String, String> properties;
public String getName() {
return name;
}
public PolicyConfiguration setName(String name) {
this.name = name;
return this;
}
public Map<String, String> getProperties() {
return properties;
}
public PolicyConfiguration setProperties(Map<String, String> properties) {
this.properties = properties;
return this;
}
}

View File

@ -0,0 +1,123 @@
/**
* 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.core.config.balancing;
import java.io.Serializable;
import java.util.Collections;
import java.util.List;
public class PoolConfiguration implements Serializable {
private String username;
private String password;
private boolean localTargetEnabled = false;
private String clusterConnection = null;
private List<String> staticConnectors = Collections.emptyList();
private String discoveryGroupName = null;
private int checkPeriod = 5000;
private int quorumSize = 1;
private int quorumTimeout = 3000;
public String getUsername() {
return username;
}
public PoolConfiguration setUsername(String username) {
this.username = username;
return this;
}
public String getPassword() {
return password;
}
public PoolConfiguration setPassword(String password) {
this.password = password;
return this;
}
public int getCheckPeriod() {
return checkPeriod;
}
public PoolConfiguration setCheckPeriod(int checkPeriod) {
this.checkPeriod = checkPeriod;
return this;
}
public int getQuorumSize() {
return quorumSize;
}
public PoolConfiguration setQuorumSize(int quorumSize) {
this.quorumSize = quorumSize;
return this;
}
public int getQuorumTimeout() {
return quorumTimeout;
}
public PoolConfiguration setQuorumTimeout(int quorumTimeout) {
this.quorumTimeout = quorumTimeout;
return this;
}
public boolean isLocalTargetEnabled() {
return localTargetEnabled;
}
public PoolConfiguration setLocalTargetEnabled(boolean localTargetEnabled) {
this.localTargetEnabled = localTargetEnabled;
return this;
}
public String getClusterConnection() {
return clusterConnection;
}
public PoolConfiguration setClusterConnection(String clusterConnection) {
this.clusterConnection = clusterConnection;
return this;
}
public List<String> getStaticConnectors() {
return staticConnectors;
}
public PoolConfiguration setStaticConnectors(List<String> staticConnectors) {
this.staticConnectors = staticConnectors;
return this;
}
public String getDiscoveryGroupName() {
return discoveryGroupName;
}
public PoolConfiguration setDiscoveryGroupName(String discoveryGroupName) {
this.discoveryGroupName = discoveryGroupName;
return this;
}
}

View File

@ -47,6 +47,7 @@ import org.apache.activemq.artemis.api.core.DiscoveryGroupConfiguration;
import org.apache.activemq.artemis.api.core.QueueConfiguration;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.core.config.balancing.BrokerBalancerConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectConfiguration;
import org.apache.activemq.artemis.core.config.BridgeConfiguration;
import org.apache.activemq.artemis.core.config.ClusterConnectionConfiguration;
@ -166,6 +167,8 @@ public class ConfigurationImpl implements Configuration, Serializable {
protected List<DivertConfiguration> divertConfigurations = new ArrayList<>();
protected List<BrokerBalancerConfiguration> brokerBalancerConfigurations = new ArrayList<>();
protected List<ClusterConnectionConfiguration> clusterConfigurations = new ArrayList<>();
protected List<AMQPBrokerConnectConfiguration> amqpBrokerConnectConfigurations = new ArrayList<>();
@ -820,6 +823,23 @@ public class ConfigurationImpl implements Configuration, Serializable {
return this;
}
@Override
public List<BrokerBalancerConfiguration> getBalancerConfigurations() {
return brokerBalancerConfigurations;
}
@Override
public ConfigurationImpl setBalancerConfigurations(final List<BrokerBalancerConfiguration> configs) {
brokerBalancerConfigurations = configs;
return this;
}
@Override
public ConfigurationImpl addBalancerConfiguration(final BrokerBalancerConfiguration config) {
brokerBalancerConfigurations.add(config);
return this;
}
@Deprecated
@Override
public List<CoreQueueConfiguration> getQueueConfigurations() {

View File

@ -18,6 +18,7 @@ package org.apache.activemq.artemis.core.config.impl;
import java.util.EnumSet;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetKey;
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
import org.apache.activemq.artemis.core.server.ComponentConfigurationRoutingType;
import org.apache.activemq.artemis.core.server.JournalType;
@ -272,4 +273,14 @@ public final class Validators {
}
}
};
public static final Validator TARGET_KEY = new Validator() {
@Override
public void validate(final String name, final Object value) {
String val = (String) value;
if (val == null || !EnumSet.allOf(TargetKey.class).contains(TargetKey.valueOf(val))) {
throw ActiveMQMessageBundle.BUNDLE.invalidTargetKey(val);
}
}
};
}

View File

@ -46,6 +46,8 @@ import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.api.core.UDPBroadcastEndpointFactory;
import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
import org.apache.activemq.artemis.core.config.balancing.BrokerBalancerConfiguration;
import org.apache.activemq.artemis.core.config.balancing.PolicyConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectConfiguration;
import org.apache.activemq.artemis.core.config.BridgeConfiguration;
import org.apache.activemq.artemis.core.config.ClusterConnectionConfiguration;
@ -62,6 +64,7 @@ import org.apache.activemq.artemis.core.config.WildcardConfiguration;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectionElement;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPBrokerConnectionAddressType;
import org.apache.activemq.artemis.core.config.amqpBrokerConnectivity.AMQPMirrorBrokerConnectionElement;
import org.apache.activemq.artemis.core.config.balancing.PoolConfiguration;
import org.apache.activemq.artemis.core.config.federation.FederationAddressPolicyConfiguration;
import org.apache.activemq.artemis.core.config.federation.FederationDownstreamConfiguration;
import org.apache.activemq.artemis.core.config.federation.FederationPolicySet;
@ -88,6 +91,8 @@ import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.core.server.ComponentConfigurationRoutingType;
import org.apache.activemq.artemis.core.server.JournalType;
import org.apache.activemq.artemis.core.server.SecuritySettingPlugin;
import org.apache.activemq.artemis.core.server.balancing.policies.PolicyFactoryResolver;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetKey;
import org.apache.activemq.artemis.core.server.cluster.impl.MessageLoadBalancingType;
import org.apache.activemq.artemis.core.server.group.impl.GroupingHandlerConfiguration;
import org.apache.activemq.artemis.core.server.metrics.ActiveMQMetricsPlugin;
@ -624,6 +629,21 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
parseDivertConfiguration(dvNode, config);
}
NodeList ccBalancers = e.getElementsByTagName("broker-balancers");
if (ccBalancers != null) {
NodeList ccBalancer = e.getElementsByTagName("broker-balancer");
if (ccBalancer != null) {
for (int i = 0; i < ccBalancer.getLength(); i++) {
Element ccNode = (Element) ccBalancer.item(i);
parseBalancerConfiguration(ccNode, config);
}
}
}
// Persistence config
config.setLargeMessagesDirectory(getString(e, "large-messages-directory", config.getLargeMessagesDirectory(), Validators.NOT_NULL_OR_EMPTY));
@ -2620,7 +2640,87 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
mainConfig.getDivertConfigurations().add(config);
}
/**
private void parseBalancerConfiguration(final Element e, final Configuration config) throws Exception {
BrokerBalancerConfiguration brokerBalancerConfiguration = new BrokerBalancerConfiguration();
brokerBalancerConfiguration.setName(e.getAttribute("name"));
brokerBalancerConfiguration.setTargetKey(TargetKey.valueOf(getString(e, "target-key", brokerBalancerConfiguration.getTargetKey().name(), Validators.TARGET_KEY)));
brokerBalancerConfiguration.setTargetKeyFilter(getString(e, "target-key-filter", brokerBalancerConfiguration.getTargetKeyFilter(), Validators.NO_CHECK));
brokerBalancerConfiguration.setLocalTargetFilter(getString(e, "local-target-filter", brokerBalancerConfiguration.getLocalTargetFilter(), Validators.NO_CHECK));
brokerBalancerConfiguration.setCacheTimeout(getInteger(e, "cache-timeout",
brokerBalancerConfiguration.getCacheTimeout(), Validators.MINUS_ONE_OR_GE_ZERO));
PolicyConfiguration policyConfiguration = null;
PoolConfiguration poolConfiguration = null;
NodeList children = e.getChildNodes();
for (int j = 0; j < children.getLength(); j++) {
Node child = children.item(j);
if (child.getNodeName().equals("policy")) {
policyConfiguration = new PolicyConfiguration();
parsePolicyConfiguration((Element)child, policyConfiguration);
brokerBalancerConfiguration.setPolicyConfiguration(policyConfiguration);
} else if (child.getNodeName().equals("pool")) {
poolConfiguration = new PoolConfiguration();
parsePoolConfiguration((Element) child, config, poolConfiguration);
brokerBalancerConfiguration.setPoolConfiguration(poolConfiguration);
}
}
config.getBalancerConfigurations().add(brokerBalancerConfiguration);
}
private void parsePolicyConfiguration(final Element e, final PolicyConfiguration policyConfiguration) throws ClassNotFoundException {
String name = e.getAttribute("name");
PolicyFactoryResolver.getInstance().resolve(name);
policyConfiguration.setName(name);
policyConfiguration.setProperties(getMapOfChildPropertyElements(e));
}
private void parsePoolConfiguration(final Element e, final Configuration config, final PoolConfiguration poolConfiguration) throws Exception {
poolConfiguration.setUsername(getString(e, "username", null, Validators.NO_CHECK));
String password = getString(e, "password", null, Validators.NO_CHECK);
poolConfiguration.setPassword(password != null ? PasswordMaskingUtil.resolveMask(
config.isMaskPassword(), password, config.getPasswordCodec()) : null);
poolConfiguration.setCheckPeriod(getInteger(e, "check-period",
poolConfiguration.getCheckPeriod(), Validators.GT_ZERO));
poolConfiguration.setQuorumSize(getInteger(e, "quorum-size",
poolConfiguration.getQuorumSize(), Validators.GT_ZERO));
poolConfiguration.setQuorumTimeout(getInteger(e, "quorum-timeout",
poolConfiguration.getQuorumTimeout(), Validators.GE_ZERO));
poolConfiguration.setLocalTargetEnabled(getBoolean(e, "local-target-enabled", poolConfiguration.isLocalTargetEnabled()));
poolConfiguration.setClusterConnection(getString(e, "cluster-connection", null, Validators.NO_CHECK));
NodeList children = e.getChildNodes();
for (int i = 0; i < children.getLength(); i++) {
Node child = children.item(i);
if (child.getNodeName().equals("discovery-group-ref")) {
poolConfiguration.setDiscoveryGroupName(child.getAttributes().getNamedItem("discovery-group-name").getNodeValue());
} else if (child.getNodeName().equals("static-connectors")) {
List<String> staticConnectorNames = new ArrayList<>();
getStaticConnectors(staticConnectorNames, child);
poolConfiguration.setStaticConnectors(staticConnectorNames);
}
}
}
/**RedirectConfiguration
* @param e
*/
protected void parseWildcardConfiguration(final Element e, final Configuration mainConfig) {

View File

@ -0,0 +1,160 @@
/**
* 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.core.management.impl;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.api.core.management.BrokerBalancerControl;
import org.apache.activemq.artemis.core.persistence.StorageManager;
import org.apache.activemq.artemis.core.server.balancing.BrokerBalancer;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.utils.JsonLoader;
import javax.json.JsonObjectBuilder;
import javax.management.MBeanAttributeInfo;
import javax.management.MBeanOperationInfo;
import javax.management.NotCompliantMBeanException;
import javax.management.openmbean.CompositeData;
import javax.management.openmbean.CompositeDataSupport;
import javax.management.openmbean.CompositeType;
import javax.management.openmbean.OpenDataException;
import javax.management.openmbean.OpenType;
import javax.management.openmbean.SimpleType;
import javax.management.openmbean.TabularData;
import javax.management.openmbean.TabularDataSupport;
import javax.management.openmbean.TabularType;
import java.util.Map;
public class BrokerBalancerControlImpl extends AbstractControl implements BrokerBalancerControl {
private final BrokerBalancer balancer;
private static CompositeType parameterType;
private static TabularType parametersType;
private static CompositeType transportConfigurationType;
private static CompositeType targetType;
public BrokerBalancerControlImpl(final BrokerBalancer balancer, final StorageManager storageManager) throws NotCompliantMBeanException {
super(BrokerBalancerControl.class, storageManager);
this.balancer = balancer;
}
@Override
public CompositeData getTarget(String key) throws Exception {
Target target = balancer.getTarget(key);
if (target != null) {
CompositeData connectorData = null;
TransportConfiguration connector = target.getConnector();
if (connector != null) {
TabularData paramsData = new TabularDataSupport(getParametersType());
for (Map.Entry<String, Object> param : connector.getParams().entrySet()) {
paramsData.put(new CompositeDataSupport(getParameterType(), new String[]{"key", "value"},
new Object[]{param.getKey(), param == null ? param : param.getValue().toString()}));
}
connectorData = new CompositeDataSupport(getTransportConfigurationType(),
new String[]{"name", "factoryClassName", "params"},
new Object[]{connector.getName(), connector.getFactoryClassName(), paramsData});
}
CompositeData targetData = new CompositeDataSupport(getTargetCompositeType(),
new String[]{"nodeID", "local", "connector"},
new Object[]{target.getNodeID(), target.isLocal(), connectorData});
return targetData;
}
return null;
}
@Override
public String getTargetAsJSON(String key) {
Target target = balancer.getTarget(key);
if (target != null) {
TransportConfiguration connector = target.getConnector();
JsonObjectBuilder targetDataBuilder = JsonLoader.createObjectBuilder()
.add("nodeID", target.getNodeID())
.add("local", target.isLocal());
if (connector == null) {
targetDataBuilder.addNull("connector");
} else {
targetDataBuilder.add("connector", connector.toJson());
}
return targetDataBuilder.build().toString();
}
return null;
}
@Override
protected MBeanOperationInfo[] fillMBeanOperationInfo() {
return MBeanInfoHelper.getMBeanOperationsInfo(BrokerBalancerControl.class);
}
@Override
protected MBeanAttributeInfo[] fillMBeanAttributeInfo() {
return MBeanInfoHelper.getMBeanAttributesInfo(BrokerBalancerControl.class);
}
private CompositeType getParameterType() throws OpenDataException {
if (parameterType == null) {
parameterType = new CompositeType("java.util.Map.Entry<java.lang.String, java.lang.String>",
"Parameter", new String[]{"key", "value"}, new String[]{"Parameter key", "Parameter value"},
new OpenType[]{SimpleType.STRING, SimpleType.STRING});
}
return parameterType;
}
private TabularType getParametersType() throws OpenDataException {
if (parametersType == null) {
parametersType = new TabularType("java.util.Map<java.lang.String, java.lang.String>",
"Parameters", getParameterType(), new String[]{"key"});
}
return parametersType;
}
private CompositeType getTransportConfigurationType() throws OpenDataException {
if (transportConfigurationType == null) {
transportConfigurationType = new CompositeType(TransportConfiguration.class.getName(),
"TransportConfiguration", new String[]{"name", "factoryClassName", "params"},
new String[]{"TransportConfiguration name", "TransportConfiguration factoryClassName", "TransportConfiguration params"},
new OpenType[]{SimpleType.STRING, SimpleType.STRING, getParametersType()});
}
return transportConfigurationType;
}
private CompositeType getTargetCompositeType() throws OpenDataException {
if (targetType == null) {
targetType = new CompositeType(Target.class.getName(),
"Target", new String[]{"nodeID", "local", "connector"},
new String[]{"Target nodeID", "Target local", "Target connector"},
new OpenType[]{SimpleType.STRING, SimpleType.BOOLEAN, getTransportConfigurationType()});
}
return targetType;
}
}

View File

@ -55,7 +55,7 @@ public class ConsumerView extends ActiveMQAbstractView<ServerConsumer> {
String consumerClientID = consumer.getConnectionClientID();
if (consumerClientID == null && session.getMetaData(ClientSession.JMS_SESSION_IDENTIFIER_PROPERTY) != null) {
//for the special case for JMS
consumerClientID = session.getMetaData("jms-client-id");
consumerClientID = session.getMetaData(ClientSession.JMS_SESSION_CLIENT_ID_PROPERTY);
}
JsonObjectBuilder obj = JsonLoader.createObjectBuilder()

View File

@ -54,7 +54,7 @@ public class ProducerView extends ActiveMQAbstractView<ServerProducer> {
String sessionClientID = session.getRemotingConnection().getClientID();
//for the special case for JMS
if (sessionClientID == null && session.getMetaData(ClientSession.JMS_SESSION_IDENTIFIER_PROPERTY) != null) {
sessionClientID = session.getMetaData("jms-client-id");
sessionClientID = session.getMetaData(ClientSession.JMS_SESSION_CLIENT_ID_PROPERTY);
}
JsonObjectBuilder obj = JsonLoader.createObjectBuilder()

View File

@ -44,6 +44,7 @@ import org.apache.activemq.artemis.core.remoting.impl.netty.HttpAcceptorHandler;
import org.apache.activemq.artemis.core.remoting.impl.netty.HttpKeepAliveRunnable;
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyAcceptor;
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnector;
import org.apache.activemq.artemis.core.remoting.impl.netty.NettySNIHostnameHandler;
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyServerConnection;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
@ -111,6 +112,7 @@ public class ProtocolHandler {
private int handshakeTimeout;
private NettySNIHostnameHandler nettySNIHostnameHandler;
ProtocolDecoder(boolean http, boolean httpEnabled) {
this.http = http;
@ -120,6 +122,8 @@ public class ProtocolHandler {
@Override
public void channelActive(ChannelHandlerContext ctx) throws Exception {
nettySNIHostnameHandler = ctx.pipeline().get(NettySNIHostnameHandler.class);
if (handshakeTimeout > 0) {
timeoutFuture = scheduledThreadPool.schedule( () -> {
ActiveMQServerLogger.LOGGER.handshakeTimeout(handshakeTimeout, nettyAcceptor.getName(), ctx.channel().remoteAddress().toString());
@ -220,6 +224,7 @@ public class ProtocolHandler {
protocolManagerToUse.addChannelHandlers(pipeline);
pipeline.addLast("handler", channelHandler);
NettyServerConnection connection = channelHandler.createConnection(ctx, protocolToUse, httpEnabled);
connection.setSNIHostname(nettySNIHostnameHandler != null ? nettySNIHostnameHandler.getHostname() : null);
protocolManagerToUse.handshake(connection, new ChannelBufferWrapper(in));
pipeline.remove(this);

View File

@ -37,6 +37,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CheckFailo
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CheckFailoverReplyMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateQueueMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionResponseMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ReattachSessionMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ReattachSessionResponseMessage;
@ -89,7 +90,8 @@ public class ActiveMQPacketHandler implements ChannelHandler {
}
switch (type) {
case PacketImpl.CREATESESSION: {
case PacketImpl.CREATESESSION:
case PacketImpl.CREATESESSION_V2: {
CreateSessionMessage request = (CreateSessionMessage) packet;
handleCreateSession(request);
@ -157,6 +159,14 @@ public class ActiveMQPacketHandler implements ChannelHandler {
ActiveMQServerLogger.LOGGER.incompatibleVersionAfterConnect(request.getVersion(), connection.getChannelVersion());
}
if (request instanceof CreateSessionMessage_V2) {
connection.setClientID(((CreateSessionMessage_V2) request).getClientID());
}
if (connection.getTransportConnection().getRedirectTo() != null) {
protocolManager.getRedirectHandler().redirect(connection, request);
}
Channel channel = connection.getChannel(request.getSessionChannelID(), request.getWindowSize());
ActiveMQPrincipal activeMQPrincipal = null;
@ -187,6 +197,8 @@ public class ActiveMQPacketHandler implements ChannelHandler {
if (e.getType() == ActiveMQExceptionType.INCOMPATIBLE_CLIENT_SERVER_VERSIONS) {
incompatibleVersion = true;
logger.debug("Sending ActiveMQException after Incompatible client", e);
} else if (e.getType() == ActiveMQExceptionType.REDIRECTED) {
logger.debug("Sending ActiveMQException after redirected client", e);
} else {
ActiveMQServerLogger.LOGGER.failedToCreateSession(e);
}

View File

@ -0,0 +1,28 @@
/**
* 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.core.protocol.core.impl;
import org.apache.activemq.artemis.core.protocol.core.CoreRemotingConnection;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage;
import org.apache.activemq.artemis.core.server.balancing.RedirectContext;
public class ActiveMQRedirectContext extends RedirectContext {
public ActiveMQRedirectContext(CoreRemotingConnection connection, CreateSessionMessage message) {
super(connection, connection.getClientID(), message.getUsername());
}
}

View File

@ -0,0 +1,52 @@
/**
* 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.core.protocol.core.impl;
import org.apache.activemq.artemis.api.core.DisconnectReason;
import org.apache.activemq.artemis.core.protocol.core.CoreRemotingConnection;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.CreateSessionMessage;
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.balancing.RedirectHandler;
public class ActiveMQRedirectHandler extends RedirectHandler<ActiveMQRedirectContext> {
public ActiveMQRedirectHandler(ActiveMQServer server) {
super(server);
}
public boolean redirect(CoreRemotingConnection connection, CreateSessionMessage message) throws Exception {
if (!connection.isVersionSupportRedirect()) {
throw ActiveMQMessageBundle.BUNDLE.incompatibleClientServer();
}
return redirect(new ActiveMQRedirectContext(connection, message));
}
@Override
public void cannotRedirect(ActiveMQRedirectContext context) throws Exception {
throw ActiveMQMessageBundle.BUNDLE.cannotRedirect();
}
@Override
public void redirectTo(ActiveMQRedirectContext context) throws Exception {
context.getConnection().disconnect(DisconnectReason.REDIRECT, context.getTarget().getNodeID(), context.getTarget().getConnector());
throw ActiveMQMessageBundle.BUNDLE.redirectConnection(context.getTarget().getConnector());
}
}

View File

@ -55,6 +55,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.ChannelImpl.CHANNEL_I
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V2;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V3;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ClusterTopologyChangeMessage_V4;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.FederationDownstreamConnectMessage;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.Ping;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SubscribeClusterTopologyUpdatesMessage;
@ -72,7 +73,7 @@ import org.apache.activemq.artemis.spi.core.remoting.Acceptor;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
import org.jboss.logging.Logger;
public class CoreProtocolManager implements ProtocolManager<Interceptor> {
public class CoreProtocolManager implements ProtocolManager<Interceptor, ActiveMQRedirectHandler> {
private static final Logger logger = Logger.getLogger(CoreProtocolManager.class);
@ -90,6 +91,8 @@ public class CoreProtocolManager implements ProtocolManager<Interceptor> {
private String securityDomain;
private final ActiveMQRedirectHandler redirectHandler;
public CoreProtocolManager(final CoreProtocolManagerFactory factory,
final ActiveMQServer server,
final List<Interceptor> incomingInterceptors,
@ -101,6 +104,8 @@ public class CoreProtocolManager implements ProtocolManager<Interceptor> {
this.incomingInterceptors = incomingInterceptors;
this.outgoingInterceptors = outgoingInterceptors;
this.redirectHandler = new ActiveMQRedirectHandler(server);
}
@Override
@ -233,6 +238,11 @@ public class CoreProtocolManager implements ProtocolManager<Interceptor> {
return securityDomain;
}
@Override
public ActiveMQRedirectHandler getRedirectHandler() {
return redirectHandler;
}
private boolean isArtemis(ActiveMQBuffer buffer) {
return buffer.getByte(0) == 'A' &&
buffer.getByte(1) == 'R' &&
@ -304,7 +314,13 @@ public class CoreProtocolManager implements ProtocolManager<Interceptor> {
entry.connectionExecutor.execute(new Runnable() {
@Override
public void run() {
if (channel0.supports(PacketImpl.CLUSTER_TOPOLOGY_V3)) {
if (channel0.supports(PacketImpl.CLUSTER_TOPOLOGY_V4)) {
channel0.send(new ClusterTopologyChangeMessage_V4(
topologyMember.getUniqueEventID(), nodeID,
topologyMember.getBackupGroupName(),
topologyMember.getScaleDownGroupName(), connectorPair,
last, server.getVersion().getIncrementingVersion()));
} else if (channel0.supports(PacketImpl.CLUSTER_TOPOLOGY_V3)) {
channel0.send(new ClusterTopologyChangeMessage_V3(
topologyMember.getUniqueEventID(), nodeID,
topologyMember.getBackupGroupName(),
@ -380,7 +396,10 @@ public class CoreProtocolManager implements ProtocolManager<Interceptor> {
String nodeId = server.getNodeID().toString();
Pair<TransportConfiguration, TransportConfiguration> emptyConfig = new Pair<>(
null, null);
if (channel0.supports(PacketImpl.CLUSTER_TOPOLOGY_V2)) {
if (channel0.supports(PacketImpl.CLUSTER_TOPOLOGY_V4)) {
channel0.send(new ClusterTopologyChangeMessage_V4(System.currentTimeMillis(), nodeId,
null, null, emptyConfig, true, server.getVersion().getIncrementingVersion()));
} else if (channel0.supports(PacketImpl.CLUSTER_TOPOLOGY_V2)) {
channel0.send(
new ClusterTopologyChangeMessage_V2(System.currentTimeMillis(),
nodeId, null, emptyConfig, true));

View File

@ -237,6 +237,8 @@ public class NettyAcceptor extends AbstractAcceptor {
private final boolean autoStart;
private final String redirectTo;
final AtomicBoolean warningPrinted = new AtomicBoolean(false);
final Executor failureExecutor;
@ -378,6 +380,8 @@ public class NettyAcceptor extends AbstractAcceptor {
connectionsAllowed = ConfigurationHelper.getLongProperty(TransportConstants.CONNECTIONS_ALLOWED, TransportConstants.DEFAULT_CONNECTIONS_ALLOWED, configuration);
autoStart = ConfigurationHelper.getBooleanProperty(TransportConstants.AUTO_START, TransportConstants.DEFAULT_AUTO_START, configuration);
redirectTo = ConfigurationHelper.getStringProperty(TransportConstants.REDIRECT_TO, TransportConstants.DEFAULT_REDIRECT_TO, configuration);
}
private Object loadSSLContext() {
@ -460,6 +464,7 @@ public class NettyAcceptor extends AbstractAcceptor {
if (sslEnabled) {
final Pair<String, Integer> peerInfo = getPeerInfo(channel);
try {
pipeline.addLast("sni", new NettySNIHostnameHandler());
pipeline.addLast("ssl", getSslHandler(channel.alloc(), peerInfo.getA(), peerInfo.getB()));
pipeline.addLast("sslHandshakeExceptionHandler", new SslHandshakeExceptionHandler());
} catch (Exception e) {
@ -930,7 +935,7 @@ public class NettyAcceptor extends AbstractAcceptor {
super.channelActive(ctx);
Listener connectionListener = new Listener();
NettyServerConnection nc = new NettyServerConnection(configuration, ctx.channel(), connectionListener, !httpEnabled && batchDelay > 0, directDeliver);
NettyServerConnection nc = new NettyServerConnection(configuration, ctx.channel(), connectionListener, !httpEnabled && batchDelay > 0, directDeliver, redirectTo);
connectionListener.connectionCreated(NettyAcceptor.this, nc, protocolHandler.getProtocol(protocol));

View File

@ -0,0 +1,42 @@
/**
* 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.core.remoting.impl.netty;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.ssl.AbstractSniHandler;
import io.netty.util.concurrent.Future;
public class NettySNIHostnameHandler extends AbstractSniHandler {
private String hostname = null;
public String getHostname() {
return hostname;
}
@Override
protected Future lookup(ChannelHandlerContext ctx, String hostname) throws Exception {
return ctx.executor().<Void>newPromise().setSuccess(null);
}
@Override
protected void onLookupComplete(ChannelHandlerContext ctx, String hostname, Future future) throws Exception {
this.hostname = hostname;
ctx.pipeline().remove(this);
}
}

View File

@ -23,12 +23,32 @@ import org.apache.activemq.artemis.spi.core.remoting.ServerConnectionLifeCycleLi
public class NettyServerConnection extends NettyConnection {
private String sniHostname;
private final String redirectTo;
public NettyServerConnection(Map<String, Object> configuration,
Channel channel,
ServerConnectionLifeCycleListener listener,
boolean batchingEnabled,
boolean directDeliver) {
boolean directDeliver,
String redirectTo) {
super(configuration, channel, listener, batchingEnabled, directDeliver);
this.redirectTo = redirectTo;
}
@Override
public String getSNIHostName() {
return sniHostname;
}
public void setSNIHostname(String sniHostname) {
this.sniHostname = sniHostname;
}
@Override
public String getRedirectTo() {
return redirectTo;
}
}

View File

@ -39,6 +39,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQInvalidTransientQueueUseExce
import org.apache.activemq.artemis.api.core.ActiveMQNonExistentQueueException;
import org.apache.activemq.artemis.api.core.ActiveMQQueueExistsException;
import org.apache.activemq.artemis.api.core.ActiveMQQueueMaxConsumerLimitReached;
import org.apache.activemq.artemis.api.core.ActiveMQRedirectedException;
import org.apache.activemq.artemis.api.core.ActiveMQReplicationTimeooutException;
import org.apache.activemq.artemis.api.core.ActiveMQSecurityException;
import org.apache.activemq.artemis.api.core.ActiveMQSessionCreationException;
@ -46,6 +47,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQUnexpectedRoutingTypeForAddr
import org.apache.activemq.artemis.api.core.DiscoveryGroupConfiguration;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.core.io.SequentialFile;
import org.apache.activemq.artemis.core.postoffice.Binding;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.ReplicationSyncFileMessage;
@ -504,4 +506,13 @@ public interface ActiveMQMessageBundle {
@Message(id = 229235, value = "Incompatible binding with name {0} already exists: {1}", format = Message.Format.MESSAGE_FORMAT)
ActiveMQIllegalStateException bindingAlreadyExists(String name, String binding);
@Message(id = 229236, value = "Invalid target key {0}", format = Message.Format.MESSAGE_FORMAT)
IllegalArgumentException invalidTargetKey(String val);
@Message(id = 229237, value = "Connection redirected to {0}", format = Message.Format.MESSAGE_FORMAT)
ActiveMQRedirectedException redirectConnection(TransportConfiguration connector);
@Message(id = 229238, value = "No target to redirect the connection")
ActiveMQRedirectedException cannotRedirect();
}

View File

@ -67,6 +67,7 @@ import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerMessagePlugi
import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerQueuePlugin;
import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerResourcePlugin;
import org.apache.activemq.artemis.core.server.plugin.ActiveMQServerSessionPlugin;
import org.apache.activemq.artemis.core.server.balancing.BrokerBalancerManager;
import org.apache.activemq.artemis.core.server.reload.ReloadManager;
import org.apache.activemq.artemis.core.settings.HierarchicalRepository;
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
@ -943,4 +944,6 @@ public interface ActiveMQServer extends ServiceComponent {
double getDiskStoreUsage();
void reloadConfigurationFile() throws Exception;
BrokerBalancerManager getBalancerManager();
}

View File

@ -40,6 +40,7 @@ import org.apache.activemq.artemis.core.persistence.OperationContext;
import org.apache.activemq.artemis.core.protocol.core.Packet;
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.BackupReplicationStartFailedMessage;
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.core.server.cluster.Bridge;
import org.apache.activemq.artemis.core.server.cluster.impl.BridgeImpl;
import org.apache.activemq.artemis.core.server.cluster.impl.ClusterConnectionImpl;
@ -47,6 +48,7 @@ import org.apache.activemq.artemis.core.server.cluster.qourum.ServerConnectVote;
import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl;
import org.apache.activemq.artemis.core.server.impl.ServerSessionImpl;
import org.apache.activemq.artemis.core.server.management.Notification;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
import org.jboss.logging.BasicLogger;
import org.jboss.logging.Logger;
import org.jboss.logging.annotations.Cause;
@ -451,6 +453,14 @@ public interface ActiveMQServerLogger extends BasicLogger {
@Message(id = 221084, value = "Requested {0} quorum votes", format = Message.Format.MESSAGE_FORMAT)
void requestedQuorumVotes(int vote);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 221085, value = "Redirect {0} to {1}", format = Message.Format.MESSAGE_FORMAT)
void redirectClientConnection(Connection connection, Target target);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 221086, value = "Cannot redirect {0}", format = Message.Format.MESSAGE_FORMAT)
void cannotRedirectClientConnection(Connection connection);
@LogMessage(level = Logger.Level.WARN)
@Message(id = 222000, value = "ActiveMQServer is being finalized and has not been stopped. Please remember to stop the server before letting it go out of scope",
format = Message.Format.MESSAGE_FORMAT)
@ -2156,4 +2166,8 @@ public interface ActiveMQServerLogger extends BasicLogger {
@LogMessage(level = Logger.Level.WARN)
@Message(id = 224108, value = "Stopped paging on address ''{0}''; size is currently: {1} bytes; max-size-bytes: {2}; global-size-bytes: {3}", format = Message.Format.MESSAGE_FORMAT)
void pageStoreStop(SimpleString storeName, long addressSize, long maxSize, long globalMaxSize);
@LogMessage(level = Logger.Level.WARN)
@Message(id = 224109, value = "BrokerBalancer {0} not found", format = Message.Format.MESSAGE_FORMAT)
void brokerBalancerNotFound(String name);
}

View File

@ -0,0 +1,193 @@
/**
* 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.core.server.balancing;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
import org.apache.activemq.artemis.core.server.ActiveMQComponent;
import org.apache.activemq.artemis.core.server.balancing.policies.Policy;
import org.apache.activemq.artemis.core.server.balancing.pools.Pool;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetKey;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetKeyResolver;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
import org.jboss.logging.Logger;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
public class BrokerBalancer implements ActiveMQComponent {
private static final Logger logger = Logger.getLogger(BrokerBalancer.class);
public static final String CLIENT_ID_PREFIX = ActiveMQDefaultConfiguration.DEFAULT_INTERNAL_NAMING_PREFIX + "balancer.client.";
private final String name;
private final TargetKey targetKey;
private final TargetKeyResolver targetKeyResolver;
private final Target localTarget;
private final Pattern localTargetFilter;
private final Pool pool;
private final Policy policy;
private final Cache<String, Target> cache;
private volatile boolean started = false;
public String getName() {
return name;
}
public TargetKey getTargetKey() {
return targetKey;
}
public Target getLocalTarget() {
return localTarget;
}
public String getLocalTargetFilter() {
return localTargetFilter != null ? localTargetFilter.pattern() : null;
}
public Pool getPool() {
return pool;
}
public Policy getPolicy() {
return policy;
}
public Cache<String, Target> getCache() {
return cache;
}
@Override
public boolean isStarted() {
return started;
}
public BrokerBalancer(final String name, final TargetKey targetKey, final String targetKeyFilter, final Target localTarget, final String localTargetFilter, final Pool pool, final Policy policy, final int cacheTimeout) {
this.name = name;
this.targetKey = targetKey;
this.targetKeyResolver = new TargetKeyResolver(targetKey, targetKeyFilter);
this.localTarget = localTarget;
this.localTargetFilter = localTargetFilter != null ? Pattern.compile(localTargetFilter) : null;
this.pool = pool;
this.policy = policy;
if (cacheTimeout == -1) {
this.cache = CacheBuilder.newBuilder().build();
} else if (cacheTimeout > 0) {
this.cache = CacheBuilder.newBuilder().expireAfterAccess(cacheTimeout, TimeUnit.MILLISECONDS).build();
} else {
this.cache = null;
}
}
@Override
public void start() throws Exception {
pool.start();
started = true;
}
@Override
public void stop() throws Exception {
started = false;
pool.stop();
}
public Target getTarget(Connection connection, String clientID, String username) {
if (clientID != null && clientID.startsWith(BrokerBalancer.CLIENT_ID_PREFIX)) {
if (logger.isDebugEnabled()) {
logger.debug("The clientID [" + clientID + "] starts with BrokerBalancer.CLIENT_ID_PREFIX");
}
return localTarget;
}
return getTarget(targetKeyResolver.resolve(connection, clientID, username));
}
public Target getTarget(String key) {
if (this.localTargetFilter != null && this.localTargetFilter.matcher(key).matches()) {
if (logger.isDebugEnabled()) {
logger.debug("The " + targetKey + "[" + key + "] matches the localTargetFilter " + localTargetFilter.pattern());
}
return localTarget;
}
Target target = null;
if (cache != null) {
target = cache.getIfPresent(key);
}
if (target != null) {
if (pool.isTargetReady(target)) {
if (logger.isDebugEnabled()) {
logger.debug("The cache returns [" + target + "] ready for " + targetKey + "[" + key + "]");
}
return target;
}
if (logger.isDebugEnabled()) {
logger.debug("The cache returns [" + target + "] not ready for " + targetKey + "[" + key + "]");
}
}
List<Target> targets = pool.getTargets();
target = policy.selectTarget(targets, key);
if (logger.isDebugEnabled()) {
logger.debug("The policy selects [" + target + "] from " + targets + " for " + targetKey + "[" + key + "]");
}
if (target != null && cache != null) {
if (logger.isDebugEnabled()) {
logger.debug("Caching " + targetKey + "[" + key + "] for [" + target + "]");
}
cache.put(key, target);
}
return target;
}
}

View File

@ -0,0 +1,191 @@
/**
* 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.core.server.balancing;
import org.apache.activemq.artemis.api.core.DiscoveryGroupConfiguration;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.core.cluster.DiscoveryGroup;
import org.apache.activemq.artemis.core.config.balancing.BrokerBalancerConfiguration;
import org.apache.activemq.artemis.core.config.balancing.PolicyConfiguration;
import org.apache.activemq.artemis.core.config.balancing.PoolConfiguration;
import org.apache.activemq.artemis.core.config.Configuration;
import org.apache.activemq.artemis.core.server.ActiveMQComponent;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.balancing.policies.Policy;
import org.apache.activemq.artemis.core.server.balancing.policies.PolicyFactory;
import org.apache.activemq.artemis.core.server.balancing.policies.PolicyFactoryResolver;
import org.apache.activemq.artemis.core.server.balancing.pools.ClusterPool;
import org.apache.activemq.artemis.core.server.balancing.pools.DiscoveryGroupService;
import org.apache.activemq.artemis.core.server.balancing.pools.DiscoveryPool;
import org.apache.activemq.artemis.core.server.balancing.pools.DiscoveryService;
import org.apache.activemq.artemis.core.server.balancing.pools.Pool;
import org.apache.activemq.artemis.core.server.balancing.pools.StaticPool;
import org.apache.activemq.artemis.core.server.balancing.targets.ActiveMQTargetFactory;
import org.apache.activemq.artemis.core.server.balancing.targets.LocalTarget;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetFactory;
import org.apache.activemq.artemis.core.server.cluster.ClusterConnection;
import org.jboss.logging.Logger;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ScheduledExecutorService;
public final class BrokerBalancerManager implements ActiveMQComponent {
private static final Logger logger = Logger.getLogger(BrokerBalancerManager.class);
private final Configuration config;
private final ActiveMQServer server;
private final ScheduledExecutorService scheduledExecutor;
private volatile boolean started = false;
private Map<String, BrokerBalancer> balancerControllers = new HashMap<>();
@Override
public boolean isStarted() {
return started;
}
public BrokerBalancerManager(final Configuration config, final ActiveMQServer server, ScheduledExecutorService scheduledExecutor) {
this.config = config;
this.server = server;
this.scheduledExecutor = scheduledExecutor;
}
public void deploy() throws Exception {
for (BrokerBalancerConfiguration balancerConfig : config.getBalancerConfigurations()) {
deployBrokerBalancer(balancerConfig);
}
}
public void deployBrokerBalancer(BrokerBalancerConfiguration config) throws Exception {
if (logger.isDebugEnabled()) {
logger.debugf("Deploying BrokerBalancer " + config.getName());
}
Target localTarget = new LocalTarget(null, server);
Pool pool = deployPool(config.getPoolConfiguration(), localTarget);
Policy policy = deployPolicy(config.getPolicyConfiguration(), pool);
BrokerBalancer balancer = new BrokerBalancer(config.getName(), config.getTargetKey(), config.getTargetKeyFilter(),
localTarget, config.getLocalTargetFilter(), pool, policy, config.getCacheTimeout());
balancerControllers.put(balancer.getName(), balancer);
server.getManagementService().registerBrokerBalancer(balancer);
}
private Pool deployPool(PoolConfiguration config, Target localTarget) throws Exception {
Pool pool;
TargetFactory targetFactory = new ActiveMQTargetFactory();
targetFactory.setUsername(config.getUsername());
targetFactory.setPassword(config.getPassword());
if (config.getClusterConnection() != null) {
ClusterConnection clusterConnection = server.getClusterManager()
.getClusterConnection(config.getClusterConnection());
pool = new ClusterPool(targetFactory, scheduledExecutor, config.getCheckPeriod(), clusterConnection);
} else if (config.getDiscoveryGroupName() != null) {
DiscoveryGroupConfiguration discoveryGroupConfiguration = server.getConfiguration().
getDiscoveryGroupConfigurations().get(config.getDiscoveryGroupName());
DiscoveryService discoveryService = new DiscoveryGroupService(new DiscoveryGroup(server.getNodeID().toString(), config.getDiscoveryGroupName(),
discoveryGroupConfiguration.getRefreshTimeout(), discoveryGroupConfiguration.getBroadcastEndpointFactory(), null));
pool = new DiscoveryPool(targetFactory, scheduledExecutor, config.getCheckPeriod(), discoveryService);
} else if (config.getStaticConnectors() != null) {
Map<String, TransportConfiguration> connectorConfigurations =
server.getConfiguration().getConnectorConfigurations();
List<TransportConfiguration> staticConnectors = new ArrayList<>();
for (String staticConnector : config.getStaticConnectors()) {
TransportConfiguration connector = connectorConfigurations.get(staticConnector);
if (connector != null) {
staticConnectors.add(connector);
} else {
logger.warn("Static connector not found: " + config.isLocalTargetEnabled());
}
}
pool = new StaticPool(targetFactory, scheduledExecutor, config.getCheckPeriod(), staticConnectors);
} else {
throw new IllegalStateException("Pool configuration not valid");
}
pool.setUsername(config.getUsername());
pool.setPassword(config.getPassword());
pool.setQuorumSize(config.getQuorumSize());
pool.setQuorumTimeout(config.getQuorumTimeout());
if (config.isLocalTargetEnabled()) {
pool.addTarget(localTarget);
}
return pool;
}
private Policy deployPolicy(PolicyConfiguration policyConfig, Pool pool) throws ClassNotFoundException {
PolicyFactory policyFactory = PolicyFactoryResolver.getInstance().resolve(policyConfig.getName());
Policy policy = policyFactory.createPolicy(policyConfig.getName());
policy.init(policyConfig.getProperties());
if (policy.getTargetProbe() != null) {
pool.addTargetProbe(policy.getTargetProbe());
}
return policy;
}
public BrokerBalancer getBalancer(String name) {
return balancerControllers.get(name);
}
@Override
public void start() throws Exception {
for (BrokerBalancer brokerBalancer : balancerControllers.values()) {
brokerBalancer.start();
}
started = true;
}
@Override
public void stop() throws Exception {
started = false;
for (BrokerBalancer balancer : balancerControllers.values()) {
balancer.stop();
server.getManagementService().unregisterBrokerBalancer(balancer.getName());
}
}
}

View File

@ -0,0 +1,57 @@
/**
* 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.core.server.balancing;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
public class RedirectContext {
private final RemotingConnection connection;
private final String clientID;
private final String username;
private Target target;
public RemotingConnection getConnection() {
return connection;
}
public String getClientID() {
return clientID;
}
public String getUsername() {
return username;
}
public Target getTarget() {
return target;
}
public void setTarget(Target target) {
this.target = target;
}
public RedirectContext(RemotingConnection connection, String clientID, String username) {
this.connection = connection;
this.clientID = clientID;
this.username = username;
}
}

View File

@ -0,0 +1,74 @@
/**
* 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.core.server.balancing;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.spi.core.remoting.Connection;
public abstract class RedirectHandler<T extends RedirectContext> {
private final ActiveMQServer server;
public ActiveMQServer getServer() {
return server;
}
protected RedirectHandler(ActiveMQServer server) {
this.server = server;
}
protected abstract void cannotRedirect(T context) throws Exception;
protected abstract void redirectTo(T context) throws Exception;
protected boolean redirect(T context) throws Exception {
Connection transportConnection = context.getConnection().getTransportConnection();
BrokerBalancer brokerBalancer = getServer().getBalancerManager().getBalancer(transportConnection.getRedirectTo());
if (brokerBalancer == null) {
ActiveMQServerLogger.LOGGER.brokerBalancerNotFound(transportConnection.getRedirectTo());
cannotRedirect(context);
return true;
}
context.setTarget(brokerBalancer.getTarget(transportConnection, context.getClientID(), context.getUsername()));
if (context.getTarget() == null) {
ActiveMQServerLogger.LOGGER.cannotRedirectClientConnection(transportConnection);
cannotRedirect(context);
return true;
}
ActiveMQServerLogger.LOGGER.redirectClientConnection(transportConnection, context.getTarget());
if (!context.getTarget().isLocal()) {
redirectTo(context);
return true;
}
return false;
}
}

View File

@ -0,0 +1,52 @@
/**
* 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.core.server.balancing.policies;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetProbe;
import java.util.Map;
public abstract class AbstractPolicy implements Policy {
private final String name;
private Map<String, String> properties;
@Override
public String getName() {
return name;
}
@Override
public Map<String, String> getProperties() {
return properties;
}
@Override
public TargetProbe getTargetProbe() {
return null;
}
@Override
public void init(Map<String, String> properties) {
this.properties = properties;
}
public AbstractPolicy(final String name) {
this.name = name;
}
}

View File

@ -0,0 +1,75 @@
/**
* 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.core.server.balancing.policies;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
public class ConsistentHashPolicy extends AbstractPolicy {
public static final String NAME = "CONSISTENT_HASH";
public ConsistentHashPolicy() {
super(NAME);
}
protected ConsistentHashPolicy(String name) {
super(name);
}
@Override
public Target selectTarget(List<Target> targets, String key) {
if (targets.size() > 1) {
NavigableMap<Integer, Target> consistentTargets = new TreeMap<>();
for (Target target : targets) {
consistentTargets.put(getHash(target.getNodeID()), target);
}
if (consistentTargets.size() > 0) {
Map.Entry<Integer, Target> consistentEntry = consistentTargets.floorEntry(getHash(key));
if (consistentEntry == null) {
consistentEntry = consistentTargets.firstEntry();
}
return consistentEntry.getValue();
}
} else if (targets.size() > 0) {
return targets.get(0);
}
return null;
}
private int getHash(String str) {
final int FNV_INIT = 0x811c9dc5;
final int FNV_PRIME = 0x01000193;
int hash = FNV_INIT;
for (int i = 0; i < str.length(); i++) {
hash = (hash ^ str.charAt(i)) * FNV_PRIME;
}
return hash;
}
}

View File

@ -0,0 +1,49 @@
/**
* 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.core.server.balancing.policies;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Supplier;
public class DefaultPolicyFactory extends PolicyFactory {
private static final Map<String, Supplier<AbstractPolicy>> supportedPolicies = new HashMap<>();
static {
supportedPolicies.put(ConsistentHashPolicy.NAME, () -> new ConsistentHashPolicy());
supportedPolicies.put(FirstElementPolicy.NAME, () -> new FirstElementPolicy());
supportedPolicies.put(LeastConnectionsPolicy.NAME, () -> new LeastConnectionsPolicy());
supportedPolicies.put(RoundRobinPolicy.NAME, () -> new RoundRobinPolicy());
}
@Override
public String[] getSupportedPolicies() {
return supportedPolicies.keySet().toArray(new String[supportedPolicies.size()]);
}
@Override
public AbstractPolicy createPolicy(String policyName) {
Supplier<AbstractPolicy> policySupplier = supportedPolicies.get(policyName);
if (policySupplier == null) {
throw new IllegalArgumentException("Policy not supported: " + policyName);
}
return policySupplier.get();
}
}

View File

@ -0,0 +1,43 @@
/**
* 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.core.server.balancing.policies;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import java.util.List;
public class FirstElementPolicy extends AbstractPolicy {
public static final String NAME = "FIRST_ELEMENT";
public FirstElementPolicy() {
super(NAME);
}
protected FirstElementPolicy(String name) {
super(name);
}
@Override
public Target selectTarget(List<Target> targets, String key) {
if (targets.size() > 0) {
return targets.get(0);
}
return null;
}
}

View File

@ -0,0 +1,133 @@
/**
* 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.core.server.balancing.policies;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetProbe;
import org.jboss.logging.Logger;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
public class LeastConnectionsPolicy extends RoundRobinPolicy {
private static final Logger logger = Logger.getLogger(LeastConnectionsPolicy.class);
public static final String NAME = "LEAST_CONNECTIONS";
public static final String UPDATE_CONNECTION_COUNT_PROBE_NAME = "UPDATE_CONNECTION_COUNT_PROBE";
public static final String CONNECTION_COUNT_THRESHOLD = "CONNECTION_COUNT_THRESHOLD";
private final Map<Target, Integer> connectionCountCache = new ConcurrentHashMap<>();
private int connectionCountThreshold = 0;
private final TargetProbe targetProbe = new TargetProbe(UPDATE_CONNECTION_COUNT_PROBE_NAME) {
@Override
public boolean check(Target target) {
try {
Integer connectionCount = target.getAttribute("broker", "ConnectionCount", Integer.class, 3000);
if (connectionCount < connectionCountThreshold) {
if (logger.isDebugEnabled()) {
logger.debug("Updating the connection count to 0/" + connectionCount + " for the target " + target);
}
connectionCount = 0;
} else if (logger.isDebugEnabled()) {
logger.debug("Updating the connection count to 0/" + connectionCount + " for the target " + target);
}
connectionCountCache.put(target, connectionCount);
return true;
} catch (Exception e) {
logger.warn("Error on updating the connectionCount for the target " + target, e);
return false;
}
}
};
@Override
public TargetProbe getTargetProbe() {
return targetProbe;
}
public LeastConnectionsPolicy() {
super(NAME);
}
@Override
public void init(Map<String, String> properties) {
super.init(properties);
if (properties != null) {
if (properties.containsKey(CONNECTION_COUNT_THRESHOLD)) {
connectionCountThreshold = Integer.valueOf(properties.get(CONNECTION_COUNT_THRESHOLD));
}
}
}
@Override
public Target selectTarget(List<Target> targets, String key) {
if (targets.size() > 1) {
NavigableMap<Integer, List<Target>> sortedTargets = new TreeMap<>();
for (Target target : targets) {
Integer connectionCount = connectionCountCache.get(target);
if (connectionCount == null) {
connectionCount = Integer.MAX_VALUE;
}
List<Target> leastTargets = sortedTargets.get(connectionCount);
if (leastTargets == null) {
leastTargets = new ArrayList<>();
sortedTargets.put(connectionCount, leastTargets);
}
leastTargets.add(target);
}
if (logger.isDebugEnabled()) {
logger.debug("LeastConnectionsPolicy.sortedTargets: " + sortedTargets);
}
List<Target> selectedTargets = sortedTargets.firstEntry().getValue();
if (selectedTargets.size() > 1) {
return super.selectTarget(selectedTargets, key);
} else {
return selectedTargets.get(0);
}
} else if (targets.size() > 0) {
return targets.get(0);
}
return null;
}
}

View File

@ -0,0 +1,36 @@
/**
* 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.core.server.balancing.policies;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetProbe;
import java.util.List;
import java.util.Map;
public interface Policy {
String getName();
TargetProbe getTargetProbe();
Map<String, String> getProperties();
void init(Map<String, String> properties);
Target selectTarget(List<Target> targets, String key);
}

View File

@ -0,0 +1,24 @@
/**
* 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.core.server.balancing.policies;
public abstract class PolicyFactory {
public abstract String[] getSupportedPolicies();
public abstract Policy createPolicy(String policyName);
}

View File

@ -0,0 +1,74 @@
/**
* 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.core.server.balancing.policies;
import org.apache.activemq.artemis.core.server.balancing.BrokerBalancer;
import java.util.HashMap;
import java.util.Map;
import java.util.ServiceLoader;
public class PolicyFactoryResolver {
private static PolicyFactoryResolver instance;
public static PolicyFactoryResolver getInstance() {
if (instance == null) {
instance = new PolicyFactoryResolver();
}
return instance;
}
private final Map<String, PolicyFactory> policyFactories = new HashMap<>();
private PolicyFactoryResolver() {
registerPolicyFactory(new DefaultPolicyFactory());
loadPolicyFactories();
}
public PolicyFactory resolve(String policyName) throws ClassNotFoundException {
PolicyFactory policyFactory = policyFactories.get(policyName);
if (policyFactory == null) {
throw new ClassNotFoundException("No PolicyFactory found for the policy " + policyName);
}
return policyFactory;
}
private void loadPolicyFactories() {
ServiceLoader<PolicyFactory> serviceLoader = ServiceLoader.load(
PolicyFactory.class, BrokerBalancer.class.getClassLoader());
for (PolicyFactory policyFactory : serviceLoader) {
registerPolicyFactory(policyFactory);
}
}
public void registerPolicyFactory(PolicyFactory policyFactory) {
for (String policyName : policyFactory.getSupportedPolicies()) {
policyFactories.put(policyName, policyFactory);
}
}
public void unregisterPolicyFactory(PolicyFactory policyFactory) {
for (String policyName : policyFactory.getSupportedPolicies()) {
policyFactories.remove(policyName, policyFactory);
}
}
}

View File

@ -0,0 +1,47 @@
/**
* 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.core.server.balancing.policies;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.utils.RandomUtil;
import java.util.List;
public class RoundRobinPolicy extends AbstractPolicy {
public static final String NAME = "ROUND_ROBIN";
private int pos = RandomUtil.randomInterval(0, Integer.MAX_VALUE);
public RoundRobinPolicy() {
super(NAME);
}
protected RoundRobinPolicy(String name) {
super(name);
}
@Override
public Target selectTarget(List<Target> targets, String key) {
if (targets.size() > 0) {
pos = pos % targets.size();
return targets.get(pos++);
}
return null;
}
}

View File

@ -0,0 +1,243 @@
/**
* 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.core.server.balancing.pools;
import org.apache.activemq.artemis.api.core.TransportConfiguration;
import org.apache.activemq.artemis.core.server.balancing.targets.Target;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetFactory;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetMonitor;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetProbe;
import org.jboss.logging.Logger;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.LockSupport;
import java.util.stream.Collectors;
public abstract class AbstractPool implements Pool {
private static final Logger logger = Logger.getLogger(AbstractPool.class);
private final TargetFactory targetFactory;
private final ScheduledExecutorService scheduledExecutor;
private final int checkPeriod;
private final List<TargetProbe> targetProbes = new ArrayList<>();
private final Map<Target, TargetMonitor> targets = new ConcurrentHashMap<>();
private final List<TargetMonitor> targetMonitors = new CopyOnWriteArrayList<>();
private String username;
private String password;
private int quorumSize;
private int quorumTimeout;
private long quorumTimeoutNanos;
private final long quorumParkNanos = TimeUnit.MILLISECONDS.toNanos(100);
private volatile boolean started = false;
@Override
public String getUsername() {
return username;
}
@Override
public void setUsername(String username) {
this.username = username;
}
@Override
public String getPassword() {
return password;
}
@Override
public void setPassword(String password) {
this.password = password;
}
@Override
public int getCheckPeriod() {
return checkPeriod;
}
@Override
public int getQuorumSize() {
return quorumSize;
}
@Override
public int getQuorumTimeout() {
return quorumTimeout;
}
@Override
public void setQuorumTimeout(int quorumTimeout) {
this.quorumTimeout = quorumTimeout;
this.quorumTimeoutNanos = TimeUnit.MILLISECONDS.toNanos(quorumTimeout);
}
@Override
public void setQuorumSize(int quorumSize) {
this.quorumSize = quorumSize;
}
@Override
public List<Target> getAllTargets() {
return targetMonitors.stream().map(targetMonitor -> targetMonitor.getTarget()).collect(Collectors.toList());
}
@Override
public List<Target> getTargets() {
List<Target> targets = targetMonitors.stream().filter(targetMonitor -> targetMonitor.isTargetReady())
.map(targetMonitor -> targetMonitor.getTarget()).collect(Collectors.toList());
if (quorumTimeout > 0 && targets.size() < quorumSize) {
final long deadline = System.nanoTime() + quorumTimeoutNanos;
while (targets.size() < quorumSize && (System.nanoTime() - deadline) < 0) {
targets = targetMonitors.stream().filter(targetMonitor -> targetMonitor.isTargetReady())
.map(targetMonitor -> targetMonitor.getTarget()).collect(Collectors.toList());
LockSupport.parkNanos(quorumParkNanos);
}
}
if (logger.isDebugEnabled()) {
logger.debugf("Ready targets are " + targets + " / " + targetMonitors + " and quorumSize is " + quorumSize);
}
return targets.size() < quorumSize ? Collections.emptyList() : targets;
}
@Override
public List<TargetProbe> getTargetProbes() {
return targetProbes;
}
@Override
public boolean isStarted() {
return started;
}
public AbstractPool(TargetFactory targetFactory, ScheduledExecutorService scheduledExecutor, int checkPeriod) {
this.targetFactory = targetFactory;
this.scheduledExecutor = scheduledExecutor;
this.checkPeriod = checkPeriod;
}
@Override
public Target getTarget(String nodeId) {
for (TargetMonitor targetMonitor : targetMonitors) {
if (nodeId.equals(targetMonitor.getTarget().getNodeID())) {
return targetMonitor.getTarget();
}
}
return null;
}
@Override
public boolean isTargetReady(Target target) {
TargetMonitor targetMonitor = targets.get(target);
return targetMonitor != null ? targetMonitor.isTargetReady() : false;
}
@Override
public void addTargetProbe(TargetProbe probe) {
targetProbes.add(probe);
}
@Override
public void removeTargetProbe(TargetProbe probe) {
targetProbes.remove(probe);
}
@Override
public void start() throws Exception {
started = true;
for (TargetMonitor targetMonitor : targetMonitors) {
targetMonitor.start();
}
}
@Override
public void stop() throws Exception {
started = false;
List<TargetMonitor> targetMonitors = new ArrayList<>(this.targetMonitors);
for (TargetMonitor targetMonitor : targetMonitors) {
removeTarget(targetMonitor.getTarget());
}
}
protected void addTarget(TransportConfiguration connector, String nodeID) {
addTarget(targetFactory.createTarget(connector, nodeID));
}
@Override
public boolean addTarget(Target target) {
TargetMonitor targetMonitor = new TargetMonitor(scheduledExecutor, checkPeriod, target, targetProbes);
if (targets.putIfAbsent(target, targetMonitor) != null) {
return false;
}
targetMonitors.add(targetMonitor);
if (started) {
targetMonitor.start();
}
return true;
}
@Override
public boolean removeTarget(Target target) {
TargetMonitor targetMonitor = targets.remove(target);
if (targetMonitor == null) {
return false;
}
targetMonitors.remove(targetMonitor);
targetMonitor.stop();
return true;
}
}

View File

@ -0,0 +1,69 @@
/**
* 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.core.server.balancing.pools;
import org.apache.activemq.artemis.api.core.client.ClusterTopologyListener;
import org.apache.activemq.artemis.api.core.client.TopologyMember;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetFactory;
import org.apache.activemq.artemis.core.server.cluster.ClusterConnection;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledExecutorService;
public class ClusterPool extends AbstractPool implements ClusterTopologyListener {
private final ClusterConnection clusterConnection;
private final Map<String, TopologyMember> clusterMembers = new ConcurrentHashMap<>();
public ClusterPool(TargetFactory targetFactory, ScheduledExecutorService scheduledExecutor,
int checkPeriod, ClusterConnection clusterConnection) {
super(targetFactory, scheduledExecutor, checkPeriod);
this.clusterConnection = clusterConnection;
}
@Override
public void start() throws Exception {
super.start();
clusterConnection.addClusterTopologyListener(this);
}
@Override
public void stop() throws Exception {
clusterConnection.removeClusterTopologyListener(this);
super.stop();
}
@Override
public void nodeUP(TopologyMember member, boolean last) {
if (!clusterConnection.getNodeID().equals(member.getNodeId()) &&
clusterMembers.putIfAbsent(member.getNodeId(), member) == null) {
addTarget(member.getLive(), member.getNodeId());
}
}
@Override
public void nodeDown(long eventUID, String nodeID) {
if (clusterMembers.remove(nodeID) != null) {
removeTarget(getTarget(nodeID));
}
}
}

View File

@ -0,0 +1,87 @@
/**
* 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.core.server.balancing.pools;
import org.apache.activemq.artemis.core.cluster.DiscoveryEntry;
import org.apache.activemq.artemis.core.cluster.DiscoveryGroup;
import org.apache.activemq.artemis.core.cluster.DiscoveryListener;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
public class DiscoveryGroupService extends DiscoveryService implements DiscoveryListener {
private final DiscoveryGroup discoveryGroup;
private final Map<String, Entry> entries = new ConcurrentHashMap<>();
public DiscoveryGroupService(DiscoveryGroup discoveryGroup) {
this.discoveryGroup = discoveryGroup;
}
@Override
public void start() throws Exception {
discoveryGroup.registerListener(this);
discoveryGroup.start();
}
@Override
public void stop() throws Exception {
discoveryGroup.unregisterListener(this);
discoveryGroup.stop();
entries.clear();
}
@Override
public boolean isStarted() {
return discoveryGroup.isStarted();
}
@Override
public void connectorsChanged(List<DiscoveryEntry> newEntries) {
Map<String, Entry> oldEntries = new HashMap<>(entries);
for (DiscoveryEntry newEntry : newEntries) {
Entry oldEntry = oldEntries.remove(newEntry.getNodeID());
if (oldEntry == null) {
Entry addingEntry = new Entry(newEntry.getNodeID(), newEntry.getConnector());
entries.put(addingEntry.getNodeID(), addingEntry);
fireEntryAddedEvent(addingEntry);
} else if (!newEntry.getConnector().equals(oldEntry.getConnector())) {
Entry updatingEntry = new Entry(newEntry.getNodeID(), newEntry.getConnector());
entries.put(updatingEntry.getNodeID(), updatingEntry);
fireEntryUpdatedEvent(oldEntry, updatingEntry);
}
}
oldEntries.forEach((nodeID, entry) -> {
entries.remove(nodeID);
fireEntryRemovedEvent(entry);
});
}
}

View File

@ -0,0 +1,70 @@
/**
* 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.core.server.balancing.pools;
import org.apache.activemq.artemis.core.server.balancing.targets.TargetFactory;
import java.util.concurrent.ScheduledExecutorService;
public class DiscoveryPool extends AbstractPool implements DiscoveryService.Listener {
private final DiscoveryService discoveryService;
public DiscoveryPool(TargetFactory targetFactory, ScheduledExecutorService scheduledExecutor,
int checkPeriod, DiscoveryService discoveryService) {
super(targetFactory, scheduledExecutor, checkPeriod);
this.discoveryService = discoveryService;
}
@Override
public void start() throws Exception {
super.start();
discoveryService.setListener(this);
discoveryService.start();
}
@Override
public void stop() throws Exception {
super.stop();
if (discoveryService != null) {
discoveryService.setListener(null);
discoveryService.stop();
}
}
@Override
public void entryAdded(DiscoveryService.Entry entry) {
addTarget(entry.getConnector(), entry.getNodeID());
}
@Override
public void entryRemoved(DiscoveryService.Entry entry) {
removeTarget(getTarget(entry.getNodeID()));
}
@Override
public void entryUpdated(DiscoveryService.Entry oldEntry, DiscoveryService.Entry newEntry) {
removeTarget(getTarget(oldEntry.getNodeID()));
addTarget(newEntry.getConnector(), newEntry.getNodeID());
}
}

Some files were not shown because too many files have changed in this diff Show More