Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id E880A200BFC for ; Fri, 9 Dec 2016 20:48:51 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id E7577160AFD; Fri, 9 Dec 2016 19:48:51 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 68683160B44 for ; Fri, 9 Dec 2016 20:48:49 +0100 (CET) Received: (qmail 91945 invoked by uid 500); 9 Dec 2016 19:48:48 -0000 Mailing-List: contact commits-help@activemq.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@activemq.apache.org Delivered-To: mailing list commits@activemq.apache.org Received: (qmail 88973 invoked by uid 99); 9 Dec 2016 19:48:46 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 09 Dec 2016 19:48:46 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 59453DFCC8; Fri, 9 Dec 2016 19:48:46 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: clebertsuconic@apache.org To: commits@activemq.apache.org Date: Fri, 09 Dec 2016 19:49:13 -0000 Message-Id: <656162a240bc40b8b99b27544af2a98c@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [29/50] [abbrv] activemq-artemis git commit: ARTEMIS-780 Added ability to define 2 Routing Types on a single addres archived-at: Fri, 09 Dec 2016 19:48:52 -0000 http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/ExpiryRunnerTest.java ---------------------------------------------------------------------- diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/ExpiryRunnerTest.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/ExpiryRunnerTest.java index bd63ca2..e678893 100644 --- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/ExpiryRunnerTest.java +++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/ExpiryRunnerTest.java @@ -32,6 +32,7 @@ import org.apache.activemq.artemis.core.config.impl.ConfigurationImpl; import org.apache.activemq.artemis.core.server.ActiveMQServer; import org.apache.activemq.artemis.core.server.ActiveMQServers; import org.apache.activemq.artemis.core.server.Queue; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.core.settings.impl.AddressSettings; import org.apache.activemq.artemis.tests.util.ActiveMQTestBase; import org.junit.Assert; @@ -71,7 +72,7 @@ public class ExpiryRunnerTest extends ActiveMQTestBase { @Test public void testExpireFromMultipleQueues() throws Exception { ClientProducer producer = clientSession.createProducer(qName); - clientSession.createQueue(qName2, qName2, null, false); + clientSession.createQueue(qName2, RoutingType.MULTICAST, qName2, null, false); AddressSettings addressSettings = new AddressSettings().setExpiryAddress(expiryAddress); server.getAddressSettingsRepository().addMatch(qName2.toString(), addressSettings); ClientProducer producer2 = clientSession.createProducer(qName2); @@ -136,8 +137,8 @@ public class ExpiryRunnerTest extends ActiveMQTestBase { AddressSettings addressSettings = new AddressSettings().setExpiryAddress(expiryAddress); server.getAddressSettingsRepository().addMatch(qName2.toString(), addressSettings); clientSession.deleteQueue(qName); - clientSession.createQueue(qName, qName, null, false); - clientSession.createQueue(qName, qName2, null, false); + clientSession.createQueue(qName, RoutingType.MULTICAST, qName, null, false); + clientSession.createQueue(qName, RoutingType.MULTICAST, qName2, null, false); ClientProducer producer = clientSession.createProducer(qName); int numMessages = 100; long expiration = System.currentTimeMillis(); @@ -241,13 +242,13 @@ public class ExpiryRunnerTest extends ActiveMQTestBase { ClientSessionFactory sessionFactory = createSessionFactory(locator); clientSession = sessionFactory.createSession(false, true, true); - clientSession.createQueue(qName, qName, null, false); + clientSession.createQueue(qName, RoutingType.MULTICAST, qName, null, false); expiryAddress = new SimpleString("EA"); expiryQueue = new SimpleString("expiryQ"); AddressSettings addressSettings = new AddressSettings().setExpiryAddress(expiryAddress); server.getAddressSettingsRepository().addMatch(qName.toString(), addressSettings); server.getAddressSettingsRepository().addMatch(qName2.toString(), addressSettings); - clientSession.createQueue(expiryAddress, expiryQueue, null, false); + clientSession.createQueue(expiryAddress, RoutingType.MULTICAST, expiryQueue, null, false); } private static class DummyMessageHandler implements Runnable { http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/PredefinedQueueTest.java ---------------------------------------------------------------------- diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/PredefinedQueueTest.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/PredefinedQueueTest.java index a3c8243..93eec9c 100644 --- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/PredefinedQueueTest.java +++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/server/PredefinedQueueTest.java @@ -33,6 +33,7 @@ import org.apache.activemq.artemis.core.config.CoreQueueConfiguration; import org.apache.activemq.artemis.core.postoffice.Bindings; import org.apache.activemq.artemis.core.server.ActiveMQServer; import org.apache.activemq.artemis.core.server.ActiveMQServers; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger; import org.apache.activemq.artemis.tests.util.ActiveMQTestBase; import org.junit.Assert; @@ -87,7 +88,7 @@ public class PredefinedQueueTest extends ActiveMQTestBase { ClientSession session = addClientSession(sf.createSession(false, true, true)); try { - session.createQueue(testAddress, queueName1, null, false); + session.createQueue(testAddress, queueName1, "", false); Assert.fail("Should throw exception"); } catch (ActiveMQQueueExistsException se) { http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTest.java ---------------------------------------------------------------------- diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTest.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTest.java index f1def68..45946ed 100644 --- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTest.java +++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTest.java @@ -17,7 +17,6 @@ package org.apache.activemq.artemis.tests.integration.stomp; import javax.jms.BytesMessage; -import javax.jms.DeliveryMode; import javax.jms.Message; import javax.jms.MessageConsumer; import javax.jms.MessageListener; @@ -25,6 +24,8 @@ import javax.jms.MessageProducer; import javax.jms.TextMessage; import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; +import java.util.HashSet; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -45,6 +46,7 @@ import org.apache.activemq.artemis.api.jms.ActiveMQJMSClient; import org.apache.activemq.artemis.core.postoffice.impl.LocalQueueBinding; import org.apache.activemq.artemis.core.protocol.stomp.Stomp; import org.apache.activemq.artemis.core.protocol.stomp.StompProtocolManagerFactory; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.core.server.Queue; import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl; import org.apache.activemq.artemis.core.server.impl.AddressInfo; @@ -269,7 +271,7 @@ public class StompTest extends StompTestBase { public void testSendMessageToNonExistentQueue() throws Exception { String nonExistentQueue = RandomUtil.randomString(); conn.connect(defUser, defPass); - send(conn, getQueuePrefix() + nonExistentQueue, null, "Hello World", true, AddressInfo.RoutingType.ANYCAST); + send(conn, getQueuePrefix() + nonExistentQueue, null, "Hello World", true, RoutingType.ANYCAST); MessageConsumer consumer = session.createConsumer(ActiveMQJMSClient.createQueue(nonExistentQueue)); TextMessage message = (TextMessage) consumer.receive(1000); @@ -300,7 +302,7 @@ public class StompTest extends StompTestBase { conn.connect(defUser, defPass); // first send a message to ensure that sending to a non-existent topic won't throw an error - send(conn, getTopicPrefix() + nonExistentTopic, null, "Hello World", true, AddressInfo.RoutingType.MULTICAST); + send(conn, getTopicPrefix() + nonExistentTopic, null, "Hello World", true, RoutingType.MULTICAST); // create a subscription on the topic and send/receive another message MessageConsumer consumer = session.createConsumer(ActiveMQJMSClient.createTopic(nonExistentTopic)); @@ -488,7 +490,7 @@ public class StompTest extends StompTestBase { Assert.assertEquals("JMSCorrelationID", "c123", message.getJMSCorrelationID()); Assert.assertEquals("getJMSType", "t345", message.getJMSType()); Assert.assertEquals("getJMSPriority", 3, message.getJMSPriority()); - Assert.assertEquals(DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); + Assert.assertEquals(javax.jms.DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); Assert.assertEquals("foo", "abc", message.getStringProperty("foo")); Assert.assertEquals("bar", "123", message.getStringProperty("bar")); @@ -527,7 +529,7 @@ public class StompTest extends StompTestBase { Assert.assertEquals("JMSCorrelationID", "c123", message.getJMSCorrelationID()); Assert.assertEquals("getJMSType", "t345", message.getJMSType()); Assert.assertEquals("getJMSPriority", 3, message.getJMSPriority()); - Assert.assertEquals(DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); + Assert.assertEquals(javax.jms.DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); Assert.assertEquals("foo", "abc", message.getStringProperty("foo")); Assert.assertEquals("longHeader", 1024, message.getStringProperty("longHeader") .length()); @@ -1257,45 +1259,45 @@ public class StompTest extends StompTestBase { @Test public void testDotAnycastPrefixOnSend() throws Exception { - testPrefix("jms.queue.", AddressInfo.RoutingType.ANYCAST, true); + testPrefix("jms.queue.", RoutingType.ANYCAST, true); } @Test public void testDotMulticastPrefixOnSend() throws Exception { - testPrefix("jms.topic.", AddressInfo.RoutingType.MULTICAST, true); + testPrefix("jms.topic.", RoutingType.MULTICAST, true); } @Test public void testDotAnycastPrefixOnSubscribe() throws Exception { - testPrefix("jms.queue.", AddressInfo.RoutingType.ANYCAST, false); + testPrefix("jms.queue.", RoutingType.ANYCAST, false); } @Test public void testDotMulticastPrefixOnSubscribe() throws Exception { - testPrefix("jms.topic.", AddressInfo.RoutingType.MULTICAST, false); + testPrefix("jms.topic.", RoutingType.MULTICAST, false); } @Test public void testSlashAnycastPrefixOnSend() throws Exception { - testPrefix("/queue/", AddressInfo.RoutingType.ANYCAST, true); + testPrefix("/queue/", RoutingType.ANYCAST, true); } @Test public void testSlashMulticastPrefixOnSend() throws Exception { - testPrefix("/topic/", AddressInfo.RoutingType.MULTICAST, true); + testPrefix("/topic/", RoutingType.MULTICAST, true); } @Test public void testSlashAnycastPrefixOnSubscribe() throws Exception { - testPrefix("/queue/", AddressInfo.RoutingType.ANYCAST, false); + testPrefix("/queue/", RoutingType.ANYCAST, false); } @Test public void testSlashMulticastPrefixOnSubscribe() throws Exception { - testPrefix("/topic/", AddressInfo.RoutingType.MULTICAST, false); + testPrefix("/topic/", RoutingType.MULTICAST, false); } - public void testPrefix(final String prefix, final AddressInfo.RoutingType routingType, final boolean send) throws Exception { + public void testPrefix(final String prefix, final RoutingType routingType, final boolean send) throws Exception { int port = 61614; final String ADDRESS = UUID.randomUUID().toString(); final String PREFIXED_ADDRESS = prefix + ADDRESS; @@ -1322,32 +1324,35 @@ public class StompTest extends StompTestBase { AddressInfo addressInfo = server.getActiveMQServer().getAddressInfo(SimpleString.toSimpleString(ADDRESS)); assertNotNull("No address was created with the name " + ADDRESS, addressInfo); - assertEquals(AddressInfo.RoutingType.valueOf(param), addressInfo.getRoutingType()); + + Set deliveryModest = new HashSet<>(); + deliveryModest.add(RoutingType.valueOf(param)); + assertEquals(deliveryModest, addressInfo.getRoutingTypes()); conn.disconnect(); } @Test public void testDotPrefixedSendAndRecieveAnycast() throws Exception { - testPrefixedSendAndRecieve("jms.queue.", AddressInfo.RoutingType.ANYCAST); + testPrefixedSendAndRecieve("jms.queue.", RoutingType.ANYCAST); } @Test public void testDotPrefixedSendAndRecieveMulticast() throws Exception { - testPrefixedSendAndRecieve("jms.topic.", AddressInfo.RoutingType.MULTICAST); + testPrefixedSendAndRecieve("jms.topic.", RoutingType.MULTICAST); } @Test public void testSlashPrefixedSendAndRecieveAnycast() throws Exception { - testPrefixedSendAndRecieve("/queue/", AddressInfo.RoutingType.ANYCAST); + testPrefixedSendAndRecieve("/queue/", RoutingType.ANYCAST); } @Test public void testSlashPrefixedSendAndRecieveMulticast() throws Exception { - testPrefixedSendAndRecieve("/topic/", AddressInfo.RoutingType.MULTICAST); + testPrefixedSendAndRecieve("/topic/", RoutingType.MULTICAST); } - public void testPrefixedSendAndRecieve(final String prefix, AddressInfo.RoutingType routingType) throws Exception { + public void testPrefixedSendAndRecieve(final String prefix, RoutingType routingType) throws Exception { int port = 61614; final String ADDRESS = UUID.randomUUID().toString(); final String PREFIXED_ADDRESS = prefix + ADDRESS; @@ -1378,12 +1383,12 @@ public class StompTest extends StompTestBase { @Test public void testMulticastOperationsOnAnycastAddress() throws Exception { - testRoutingSemantics(AddressInfo.RoutingType.MULTICAST.toString(), getQueuePrefix() + getQueueName()); + testRoutingSemantics(RoutingType.MULTICAST.toString(), getQueuePrefix() + getQueueName()); } @Test public void testAnycastOperationsOnMulticastAddress() throws Exception { - testRoutingSemantics(AddressInfo.RoutingType.ANYCAST.toString(), getTopicPrefix() + getTopicName()); + testRoutingSemantics(RoutingType.ANYCAST.toString(), getTopicPrefix() + getTopicName()); } public void testRoutingSemantics(String routingType, String destination) throws Exception { @@ -1402,7 +1407,7 @@ public class StompTest extends StompTestBase { uuid = UUID.randomUUID().toString(); frame = conn.createFrame(Stomp.Commands.SEND) - .addHeader(Stomp.Headers.Send.DESTINATION_TYPE, AddressInfo.RoutingType.MULTICAST.toString()) + .addHeader(Stomp.Headers.Send.DESTINATION_TYPE, RoutingType.MULTICAST.toString()) .addHeader(Stomp.Headers.Send.DESTINATION, getQueuePrefix() + getQueueName()) .addHeader(Stomp.Headers.RECEIPT_REQUESTED, uuid); http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTestBase.java ---------------------------------------------------------------------- diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTestBase.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTestBase.java index 2cde849..30cf841 100644 --- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTestBase.java +++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/StompTestBase.java @@ -46,7 +46,7 @@ import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants; import org.apache.activemq.artemis.core.security.Role; import org.apache.activemq.artemis.core.server.ActiveMQServer; import org.apache.activemq.artemis.core.server.ActiveMQServers; -import org.apache.activemq.artemis.core.server.impl.AddressInfo; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory; import org.apache.activemq.artemis.jms.client.ActiveMQJMSConnectionFactory; import org.apache.activemq.artemis.jms.server.JMSServerManager; @@ -363,7 +363,7 @@ public abstract class StompTestBase extends ActiveMQTestBase { String destination, boolean receipt) throws IOException, InterruptedException { ClientStompFrame frame = conn.createFrame(Stomp.Commands.SUBSCRIBE) - .addHeader(Stomp.Headers.Subscribe.SUBSCRIPTION_TYPE, AddressInfo.RoutingType.ANYCAST.toString()) + .addHeader(Stomp.Headers.Subscribe.SUBSCRIPTION_TYPE, RoutingType.ANYCAST.toString()) .addHeader(Stomp.Headers.Subscribe.DESTINATION, destination); if (subscriptionId != null) { frame.addHeader(Stomp.Headers.Subscribe.ID, subscriptionId); @@ -413,7 +413,7 @@ public abstract class StompTestBase extends ActiveMQTestBase { boolean receipt, boolean noLocal) throws IOException, InterruptedException { ClientStompFrame frame = conn.createFrame(Stomp.Commands.SUBSCRIBE) - .addHeader(Stomp.Headers.Subscribe.SUBSCRIPTION_TYPE, AddressInfo.RoutingType.MULTICAST.toString()) + .addHeader(Stomp.Headers.Subscribe.SUBSCRIPTION_TYPE, RoutingType.MULTICAST.toString()) .addHeader(Stomp.Headers.Subscribe.DESTINATION, getTopicPrefix() + getTopicName()); if (subscriptionId != null) { frame.addHeader(Stomp.Headers.Subscribe.ID, subscriptionId); @@ -492,11 +492,11 @@ public abstract class StompTestBase extends ActiveMQTestBase { return send(conn, destination, contentType, body, receipt, null); } - public ClientStompFrame send(StompClientConnection conn, String destination, String contentType, String body, boolean receipt, AddressInfo.RoutingType destinationType) throws IOException, InterruptedException { + public ClientStompFrame send(StompClientConnection conn, String destination, String contentType, String body, boolean receipt, RoutingType destinationType) throws IOException, InterruptedException { return send(conn, destination, contentType, body, receipt, destinationType, null); } - public ClientStompFrame send(StompClientConnection conn, String destination, String contentType, String body, boolean receipt, AddressInfo.RoutingType destinationType, String txId) throws IOException, InterruptedException { + public ClientStompFrame send(StompClientConnection conn, String destination, String contentType, String body, boolean receipt, RoutingType destinationType, String txId) throws IOException, InterruptedException { ClientStompFrame frame = conn.createFrame(Stomp.Commands.SEND) .addHeader(Stomp.Headers.Send.DESTINATION, destination) .setBody(body); http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v11/StompV11Test.java ---------------------------------------------------------------------- diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v11/StompV11Test.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v11/StompV11Test.java index d4158ac..01f1cf8 100644 --- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v11/StompV11Test.java +++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v11/StompV11Test.java @@ -17,7 +17,6 @@ package org.apache.activemq.artemis.tests.integration.stomp.v11; import javax.jms.BytesMessage; -import javax.jms.DeliveryMode; import javax.jms.Message; import javax.jms.MessageConsumer; import javax.jms.MessageListener; @@ -32,7 +31,7 @@ import java.util.concurrent.TimeUnit; import org.apache.activemq.artemis.api.core.SimpleString; import org.apache.activemq.artemis.core.protocol.stomp.Stomp; -import org.apache.activemq.artemis.core.server.impl.AddressInfo; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger; import org.apache.activemq.artemis.tests.integration.stomp.StompTestBase; import org.apache.activemq.artemis.tests.integration.stomp.util.ClientStompFrame; @@ -1757,7 +1756,7 @@ public class StompV11Test extends StompTestBase { Assert.assertEquals("JMSCorrelationID", "c123", message.getJMSCorrelationID()); Assert.assertEquals("getJMSType", "t345", message.getJMSType()); Assert.assertEquals("getJMSPriority", 3, message.getJMSPriority()); - Assert.assertEquals(DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); + Assert.assertEquals(javax.jms.DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); Assert.assertEquals("foo", "abc", message.getStringProperty("foo")); Assert.assertEquals("bar", "123", message.getStringProperty("bar")); @@ -1796,7 +1795,7 @@ public class StompV11Test extends StompTestBase { Assert.assertEquals("JMSCorrelationID", "c123", message.getJMSCorrelationID()); Assert.assertEquals("getJMSType", "t345", message.getJMSType()); Assert.assertEquals("getJMSPriority", 3, message.getJMSPriority()); - Assert.assertEquals(DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); + Assert.assertEquals(javax.jms.DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); Assert.assertEquals("foo", "abc", message.getStringProperty("foo")); Assert.assertEquals("longHeader", 2048, message.getStringProperty("longHeader").length()); @@ -2163,7 +2162,7 @@ public class StompV11Test extends StompTestBase { public void testSendMessageToNonExistentQueueWithAutoCreation() throws Exception { conn.connect(defUser, defPass); - send(conn, "NonExistentQueue" + UUID.randomUUID().toString(), null, "Hello World", true, AddressInfo.RoutingType.ANYCAST); + send(conn, "NonExistentQueue" + UUID.randomUUID().toString(), null, "Hello World", true, RoutingType.ANYCAST); conn.disconnect(); } http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v12/StompV12Test.java ---------------------------------------------------------------------- diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v12/StompV12Test.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v12/StompV12Test.java index 09fb0ba..0957eb2 100644 --- a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v12/StompV12Test.java +++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/stomp/v12/StompV12Test.java @@ -17,7 +17,6 @@ package org.apache.activemq.artemis.tests.integration.stomp.v12; import javax.jms.BytesMessage; -import javax.jms.DeliveryMode; import javax.jms.JMSException; import javax.jms.Message; import javax.jms.MessageConsumer; @@ -33,7 +32,7 @@ import java.util.concurrent.TimeUnit; import org.apache.activemq.artemis.api.core.SimpleString; import org.apache.activemq.artemis.core.protocol.stomp.Stomp; -import org.apache.activemq.artemis.core.server.impl.AddressInfo; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.core.settings.impl.AddressSettings; import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger; import org.apache.activemq.artemis.tests.integration.stomp.StompTestBase; @@ -1755,7 +1754,7 @@ public class StompV12Test extends StompTestBase { Assert.assertEquals("JMSCorrelationID", "c123", message.getJMSCorrelationID()); Assert.assertEquals("getJMSType", "t345", message.getJMSType()); Assert.assertEquals("getJMSPriority", 3, message.getJMSPriority()); - Assert.assertEquals(DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); + Assert.assertEquals(javax.jms.DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); Assert.assertEquals("foo", "abc", message.getStringProperty("foo")); Assert.assertEquals("bar", "123", message.getStringProperty("bar")); @@ -1794,7 +1793,7 @@ public class StompV12Test extends StompTestBase { Assert.assertEquals("JMSCorrelationID", "c123", message.getJMSCorrelationID()); Assert.assertEquals("getJMSType", "t345", message.getJMSType()); Assert.assertEquals("getJMSPriority", 3, message.getJMSPriority()); - Assert.assertEquals(DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); + Assert.assertEquals(javax.jms.DeliveryMode.PERSISTENT, message.getJMSDeliveryMode()); Assert.assertEquals("foo", "abc", message.getStringProperty("foo")); Assert.assertEquals("very-very-long-stomp-message-header", 2048, message.getStringProperty("very-very-long-stomp-message-header").length()); @@ -2207,7 +2206,7 @@ public class StompV12Test extends StompTestBase { public void testSendMessageToNonExistentQueueWithAutoCreation() throws Exception { conn.connect(defUser, defPass); - send(conn, "NonExistentQueue" + UUID.randomUUID().toString(), null, "Hello World", true, AddressInfo.RoutingType.ANYCAST); + send(conn, "NonExistentQueue" + UUID.randomUUID().toString(), null, "Hello World", true, RoutingType.ANYCAST); conn.disconnect(); } http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/message/MessageHeaderTest.java ---------------------------------------------------------------------- diff --git a/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/message/MessageHeaderTest.java b/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/message/MessageHeaderTest.java index 6d70569..90b4d3e 100644 --- a/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/message/MessageHeaderTest.java +++ b/tests/jms-tests/src/test/java/org/apache/activemq/artemis/jms/tests/message/MessageHeaderTest.java @@ -49,6 +49,7 @@ import org.apache.activemq.artemis.api.core.client.SendAcknowledgementHandler; import org.apache.activemq.artemis.api.core.client.SessionFailureListener; import org.apache.activemq.artemis.core.client.impl.ClientMessageImpl; import org.apache.activemq.artemis.core.remoting.FailureListener; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.jms.client.ActiveMQBytesMessage; import org.apache.activemq.artemis.jms.client.ActiveMQMapMessage; import org.apache.activemq.artemis.jms.client.ActiveMQMessage; @@ -867,6 +868,218 @@ public class MessageHeaderTest extends MessageHeaderTestBase { final String filter) throws ActiveMQException { } + /** + * Creates a non-temporary queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param durable whether the queue is durable or not + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(SimpleString address, RoutingType routingType, SimpleString queueName, boolean durable) throws ActiveMQException { + + } + + /** + * Creates a transient queue. A queue that will exist as long as there are consumers. When the last consumer is closed the queue will be deleted + *

+ * Notice: you will get an exception if the address or the filter doesn't match to an already existent queue + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param durable if the queue is durable + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createSharedQueue(SimpleString address, RoutingType routingType, SimpleString queueName, boolean durable) throws ActiveMQException { + + } + + /** + * Creates a transient queue. A queue that will exist as long as there are consumers. When the last consumer is closed the queue will be deleted + *

+ * Notice: you will get an exception if the address or the filter doesn't match to an already existent queue + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param filter whether the queue is durable or not + * @param durable if the queue is durable + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createSharedQueue(SimpleString address, RoutingType routingType, SimpleString queueName, SimpleString filter, + boolean durable) throws ActiveMQException { + + } + + /** + * Creates a non-temporary queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param durable whether the queue is durable or not + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(String address, RoutingType routingType, String queueName, boolean durable) throws ActiveMQException { + + } + + /** + * Creates a non-temporary queue non-durable queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(String address, RoutingType routingType, String queueName) throws ActiveMQException { + + } + + /** + * Creates a non-temporary queue non-durable queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(SimpleString address, RoutingType routingType, SimpleString queueName) throws ActiveMQException { + + } + + /** + * Creates a non-temporary queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param filter only messages which match this filter will be put in the queue + * @param durable whether the queue is durable or not + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(SimpleString address, RoutingType routingType, SimpleString queueName, SimpleString filter, + boolean durable) throws ActiveMQException { + + } + + /** + * Creates a non-temporaryqueue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param filter only messages which match this filter will be put in the queue + * @param durable whether the queue is durable or not + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(String address, RoutingType routingType, String queueName, String filter, boolean durable) throws ActiveMQException { + + } + + /** + * Creates a non-temporary queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param filter only messages which match this filter will be put in the queue + * @param durable whether the queue is durable or not + * @param autoCreated whether to mark this queue as autoCreated or not + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(SimpleString address, RoutingType routingType, SimpleString queueName, SimpleString filter, + boolean durable, + boolean autoCreated) throws ActiveMQException { + + } + + /** + * Creates a non-temporaryqueue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param filter only messages which match this filter will be put in the queue + * @param durable whether the queue is durable or not + * @param autoCreated whether to mark this queue as autoCreated or not + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createQueue(String address, RoutingType routingType, String queueName, String filter, + boolean durable, + boolean autoCreated) throws ActiveMQException { + + } + + /** + * Creates a temporary queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createTemporaryQueue(SimpleString address, RoutingType routingType, SimpleString queueName) throws ActiveMQException { + + } + + /** + * Creates a temporary queue. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createTemporaryQueue(String address, RoutingType routingType, String queueName) throws ActiveMQException { + + } + + /** + * Creates a temporary queue with a filter. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param filter only messages which match this filter will be put in the queue + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createTemporaryQueue(SimpleString address, + RoutingType routingType, + SimpleString queueName, + SimpleString filter) throws ActiveMQException { + + } + + /** + * Creates a temporary queue with a filter. + * + * @param address the queue will be bound to this address + * @param routingType the delivery mode for this queue, MULTICAST or ANYCAST + * @param queueName the name of the queue + * @param filter only messages which match this filter will be put in the queue + * @throws ActiveMQException in an exception occurs while creating the queue + */ + @Override + public void createTemporaryQueue(String address, RoutingType routingType, String queueName, String filter) throws ActiveMQException { + + } + @Override public void deleteQueue(final SimpleString queueName) throws ActiveMQException { } @@ -1130,8 +1343,32 @@ public class MessageHeaderTest extends MessageHeaderTestBase { return 0; } + /** + * Create Address with a single initial routing type + * + * @param address + * @param routingTypes + * @param autoCreated @throws ActiveMQException + */ + @Override + public void createAddress(SimpleString address, + Set routingTypes, + boolean autoCreated) throws ActiveMQException { + + } + + /** + * Create Address with a single initial routing type + * + * @param address + * @param routingType + * @param autoCreated + * @throws ActiveMQException + */ @Override - public void createAddress(SimpleString address, boolean multicast, final boolean autoCreated) throws ActiveMQException { + public void createAddress(SimpleString address, + RoutingType routingType, + boolean autoCreated) throws ActiveMQException { } http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/joram-tests/src/test/java/org/apache/activemq/artemis/common/AbstractAdmin.java ---------------------------------------------------------------------- diff --git a/tests/joram-tests/src/test/java/org/apache/activemq/artemis/common/AbstractAdmin.java b/tests/joram-tests/src/test/java/org/apache/activemq/artemis/common/AbstractAdmin.java index f76c5b5..bbe99fc 100644 --- a/tests/joram-tests/src/test/java/org/apache/activemq/artemis/common/AbstractAdmin.java +++ b/tests/joram-tests/src/test/java/org/apache/activemq/artemis/common/AbstractAdmin.java @@ -34,7 +34,7 @@ import org.apache.activemq.artemis.api.core.client.ServerLocator; import org.apache.activemq.artemis.api.core.management.ManagementHelper; import org.apache.activemq.artemis.api.core.management.ResourceNames; import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnectorFactory; -import org.apache.activemq.artemis.core.server.impl.AddressInfo; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.tests.util.SpawnedVMSupport; import org.objectweb.jtests.jms.admin.Admin; @@ -150,7 +150,7 @@ public class AbstractAdmin implements Admin { public void createTopic(final String name) { Boolean result; try { - invokeSyncOperation(ResourceNames.BROKER, "createAddress", name, (int)AddressInfo.RoutingType.MULTICAST.getType(), false, -1); + invokeSyncOperation(ResourceNames.BROKER, "createAddress", name, (int) RoutingType.MULTICAST.getType(), false, -1); } catch (Exception e) { throw new IllegalStateException(e); } http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/MessageDefaultTest.java ---------------------------------------------------------------------- diff --git a/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/MessageDefaultTest.java b/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/MessageDefaultTest.java index 5beb7c0..ca3ede0 100644 --- a/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/MessageDefaultTest.java +++ b/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/MessageDefaultTest.java @@ -30,7 +30,7 @@ import org.objectweb.jtests.jms.framework.JMSTestCase; public class MessageDefaultTest extends JMSTestCase { /** - * test that the DEFAULT_DELIVERY_MODE of javax.jms.Message + * test that the DEFAULT_ROUTING_TYPE of javax.jms.Message * corresponds to javax.jms.Delivery.PERSISTENT. */ @Test http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/headers/MessageHeaderTest.java ---------------------------------------------------------------------- diff --git a/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/headers/MessageHeaderTest.java b/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/headers/MessageHeaderTest.java index 1317ff8..a58db63 100644 --- a/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/headers/MessageHeaderTest.java +++ b/tests/joram-tests/src/test/java/org/objectweb/jtests/jms/conform/message/headers/MessageHeaderTest.java @@ -133,12 +133,12 @@ public class MessageHeaderTest extends PTPTestCase { /** * Test that the JMSDeliveryMode header field value is ignored * when the message is sent and that it holds the value specified by the sending - * method (i.e. Message.DEFAULT_DELIVERY_MODE in this test when the message is received. + * method (i.e. Message.DEFAULT_ROUTING_TYPE in this test when the message is received. */ @Test public void testJMSDeliveryMode() { try { - // sender has been created with the DEFAULT_DELIVERY_MODE which is PERSISTENT + // sender has been created with the DEFAULT_ROUTING_TYPE which is PERSISTENT Assert.assertEquals(DeliveryMode.PERSISTENT, sender.getDeliveryMode()); Message message = senderSession.createMessage(); // send a message specfiying NON_PERSISTENT for the JMSDeliveryMode header field http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/7a51491c/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/postoffice/impl/FakeQueue.java ---------------------------------------------------------------------- diff --git a/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/postoffice/impl/FakeQueue.java b/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/postoffice/impl/FakeQueue.java index ef5c05e..ed15c71 100644 --- a/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/postoffice/impl/FakeQueue.java +++ b/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/postoffice/impl/FakeQueue.java @@ -25,6 +25,7 @@ import org.apache.activemq.artemis.api.core.SimpleString; import org.apache.activemq.artemis.core.filter.Filter; import org.apache.activemq.artemis.core.paging.cursor.PageSubscription; import org.apache.activemq.artemis.core.server.Consumer; +import org.apache.activemq.artemis.core.server.RoutingType; import org.apache.activemq.artemis.core.server.MessageReference; import org.apache.activemq.artemis.core.server.Queue; import org.apache.activemq.artemis.core.server.RoutingContext; @@ -559,6 +560,16 @@ public class FakeQueue implements Queue { return subs; } + @Override + public RoutingType getRoutingType() { + return null; + } + + @Override + public void setRoutingType(RoutingType routingType) { + + } + public void setPageSubscription(PageSubscription sub) { this.subs = sub; }