You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ma...@apache.org on 2016/11/01 10:22:02 UTC

[33/34] activemq-artemis git commit: ARTEMIS-813 Load AddressInfo into AddressManager

ARTEMIS-813 Load AddressInfo into AddressManager


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/54ad2bc7
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/54ad2bc7
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/54ad2bc7

Branch: refs/heads/ARTEMIS-780
Commit: 54ad2bc72309e10addc9d0a8e430ff9e3c4b816b
Parents: af5f1b1
Author: Martyn Taylor <mt...@redhat.com>
Authored: Wed Oct 19 19:16:31 2016 +0100
Committer: Martyn Taylor <mt...@redhat.com>
Committed: Tue Nov 1 10:20:52 2016 +0000

----------------------------------------------------------------------
 .../core/config/CoreAddressConfiguration.java   |  6 +-
 .../deployers/impl/FileConfigurationParser.java |  3 +-
 .../artemis/core/postoffice/AddressManager.java |  7 ++
 .../artemis/core/postoffice/PostOffice.java     |  8 +++
 .../core/postoffice/impl/PostOfficeImpl.java    | 16 +++++
 .../postoffice/impl/SimpleAddressManager.java   | 18 ++++++
 .../artemis/core/server/ActiveMQServer.java     | 13 +++-
 .../core/server/impl/ActiveMQServerImpl.java    | 49 ++++++++++++--
 .../artemis/core/server/impl/AddressInfo.java   | 67 ++++++++++++++++++++
 .../core/config/impl/FileConfigurationTest.java |  5 +-
 .../integration/addressing/AddressingTest.java  | 21 ++++++
 .../core/server/impl/fakes/FakePostOffice.java  | 17 +++++
 12 files changed, 213 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/config/CoreAddressConfiguration.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/config/CoreAddressConfiguration.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/config/CoreAddressConfiguration.java
index cb6d43f..e01c398 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/config/CoreAddressConfiguration.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/config/CoreAddressConfiguration.java
@@ -21,14 +21,10 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo.RoutingType;
 
 public class CoreAddressConfiguration implements Serializable {
 
-   public enum RoutingType {
-      MULTICAST,
-      ANYCAST
-   }
-
    private String name = null;
 
    private RoutingType routingType = null;

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
index fc045b3..2dccb03 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/deployers/impl/FileConfigurationParser.java
@@ -64,6 +64,7 @@ import org.apache.activemq.artemis.core.server.JournalType;
 import org.apache.activemq.artemis.core.server.SecuritySettingPlugin;
 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.impl.AddressInfo;
 import org.apache.activemq.artemis.core.settings.impl.AddressFullMessagePolicy;
 import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
 import org.apache.activemq.artemis.core.settings.impl.ResourceLimitSettings;
@@ -895,7 +896,7 @@ public final class FileConfigurationParser extends XMLConfigurationUtil {
 
       CoreAddressConfiguration addressConfiguration = new CoreAddressConfiguration();
       addressConfiguration.setName(name)
-         .setRoutingType(CoreAddressConfiguration.RoutingType.valueOf(routingType.toUpperCase()));
+         .setRoutingType(AddressInfo.RoutingType.valueOf(routingType.toUpperCase()));
 
       NodeList children = node.getChildNodes();
       for (int j = 0; j < children.getLength(); j++) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/AddressManager.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/AddressManager.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/AddressManager.java
index efc1297..5519822 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/AddressManager.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/AddressManager.java
@@ -20,6 +20,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.activemq.artemis.api.core.SimpleString;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.transaction.Transaction;
 
 /**
@@ -50,4 +51,10 @@ public interface AddressManager {
    Map<SimpleString, Binding> getBindings();
 
    Set<SimpleString> getAddresses();
+
+   AddressInfo addAddressInfo(AddressInfo addressInfo);
+
+   AddressInfo removeAddressInfo(SimpleString address);
+
+   AddressInfo getAddressInfo(SimpleString address);
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/PostOffice.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/PostOffice.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/PostOffice.java
index 4c0c4b0..f719966 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/PostOffice.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/PostOffice.java
@@ -27,6 +27,7 @@ import org.apache.activemq.artemis.core.server.Queue;
 import org.apache.activemq.artemis.core.server.QueueCreator;
 import org.apache.activemq.artemis.core.server.RoutingContext;
 import org.apache.activemq.artemis.core.server.ServerMessage;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.transaction.Transaction;
 
 /**
@@ -42,6 +43,12 @@ import org.apache.activemq.artemis.core.transaction.Transaction;
  */
 public interface PostOffice extends ActiveMQComponent {
 
+   AddressInfo addAddressInfo(AddressInfo addressInfo);
+
+   AddressInfo removeAddressInfo(SimpleString address);
+
+   AddressInfo getAddressInfo(SimpleString address);
+
    void addBinding(Binding binding) throws Exception;
 
    Binding removeBinding(SimpleString uniqueName, Transaction tx, boolean deleteData) throws Exception;
@@ -113,4 +120,5 @@ public interface PostOffice extends ActiveMQComponent {
 
    Set<SimpleString> getAddresses();
 
+
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/PostOfficeImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/PostOfficeImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/PostOfficeImpl.java
index cc06603..9b7ed0c 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/PostOfficeImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/PostOfficeImpl.java
@@ -69,6 +69,7 @@ import org.apache.activemq.artemis.core.server.RouteContextList;
 import org.apache.activemq.artemis.core.server.RoutingContext;
 import org.apache.activemq.artemis.core.server.ServerMessage;
 import org.apache.activemq.artemis.core.server.group.GroupingHandler;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.server.impl.RoutingContextImpl;
 import org.apache.activemq.artemis.core.server.impl.ServerMessageImpl;
 import org.apache.activemq.artemis.core.server.management.ManagementService;
@@ -418,6 +419,21 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
 
    // PostOffice implementation -----------------------------------------------
 
+   @Override
+   public AddressInfo addAddressInfo(AddressInfo addressInfo) {
+      return addressManager.addAddressInfo(addressInfo);
+   }
+
+   @Override
+   public AddressInfo removeAddressInfo(SimpleString address) {
+      return addressManager.removeAddressInfo(address);
+   }
+
+   @Override
+   public AddressInfo getAddressInfo(SimpleString addressName) {
+      return addressManager.getAddressInfo(addressName);
+   }
+
    // TODO - needs to be synchronized to prevent happening concurrently with activate()
    // (and possible removeBinding and other methods)
    // Otherwise can have situation where createQueue comes in before failover, then failover occurs

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/SimpleAddressManager.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/SimpleAddressManager.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/SimpleAddressManager.java
index 9f26b0b..2994f9e 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/SimpleAddressManager.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/postoffice/impl/SimpleAddressManager.java
@@ -30,6 +30,7 @@ import org.apache.activemq.artemis.core.postoffice.Binding;
 import org.apache.activemq.artemis.core.postoffice.Bindings;
 import org.apache.activemq.artemis.core.postoffice.BindingsFactory;
 import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.transaction.Transaction;
 import org.jboss.logging.Logger;
 
@@ -40,6 +41,8 @@ public class SimpleAddressManager implements AddressManager {
 
    private static final Logger logger = Logger.getLogger(Page.class);
 
+   private final ConcurrentMap<SimpleString, AddressInfo> addressInfoMap = new ConcurrentHashMap<>();
+
    /**
     * HashMap<Address, Binding>
     */
@@ -178,4 +181,19 @@ public class SimpleAddressManager implements AddressManager {
 
       return prevBindings != null;
    }
+
+   @Override
+   public AddressInfo addAddressInfo(AddressInfo addressInfo) {
+      return addressInfoMap.putIfAbsent(addressInfo.getName(), addressInfo);
+   }
+
+   @Override
+   public AddressInfo removeAddressInfo(SimpleString address) {
+      return addressInfoMap.remove(address);
+   }
+
+   @Override
+   public AddressInfo getAddressInfo(SimpleString addressName) {
+      return addressInfoMap.get(addressName);
+   }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
index a43fec8..fb5aee4 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/ActiveMQServer.java
@@ -39,6 +39,7 @@ import org.apache.activemq.artemis.core.server.cluster.ClusterManager;
 import org.apache.activemq.artemis.core.server.cluster.ha.HAPolicy;
 import org.apache.activemq.artemis.core.server.group.GroupingHandler;
 import org.apache.activemq.artemis.core.server.impl.Activation;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.server.impl.ConnectorsService;
 import org.apache.activemq.artemis.core.server.management.ManagementService;
 import org.apache.activemq.artemis.core.server.reload.ReloadManager;
@@ -377,10 +378,12 @@ public interface ActiveMQServer extends ActiveMQComponent {
 
    void stop(boolean failoverOnServerShutdown) throws Exception;
 
+   AddressInfo getAddressInfo(SimpleString address);
+
    /*
-   * add a ProtocolManagerFactory to be used. Note if @see Configuration#isResolveProtocols is tur then this factory will
-   * replace any factories with the same protocol
-   * */
+      * add a ProtocolManagerFactory to be used. Note if @see Configuration#isResolveProtocols is tur then this factory will
+      * replace any factories with the same protocol
+      * */
    void addProtocolManagerFactory(ProtocolManagerFactory factory);
 
    /*
@@ -409,4 +412,8 @@ public interface ActiveMQServer extends ActiveMQComponent {
    boolean addClientConnection(String clientId, boolean unique);
 
    void removeClientConnection(String clientId);
+
+   AddressInfo addAddressInfo(AddressInfo addressInfo);
+
+   AddressInfo removeAddressInfo(SimpleString address);
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
index d2de964..208a317 100644
--- a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ActiveMQServerImpl.java
@@ -55,6 +55,7 @@ import org.apache.activemq.artemis.core.client.impl.ClientSessionFactoryImpl;
 import org.apache.activemq.artemis.core.config.BridgeConfiguration;
 import org.apache.activemq.artemis.core.config.Configuration;
 import org.apache.activemq.artemis.core.config.ConfigurationUtils;
+import org.apache.activemq.artemis.core.config.CoreAddressConfiguration;
 import org.apache.activemq.artemis.core.config.CoreQueueConfiguration;
 import org.apache.activemq.artemis.core.config.DivertConfiguration;
 import org.apache.activemq.artemis.core.config.StoreConfiguration;
@@ -2000,6 +2001,9 @@ public class ActiveMQServerImpl implements ActiveMQServer {
 
       // Deploy the rest of the stuff
 
+      // Deploy predefined addresses
+      deployAddressesFromConfiguration();
+
       // Deploy any predefined queues
       deployQueuesFromConfiguration();
 
@@ -2078,11 +2082,26 @@ public class ActiveMQServerImpl implements ActiveMQServer {
       }
    }
 
-   private void deployQueuesFromConfiguration() throws Exception {
-      for (CoreQueueConfiguration config : configuration.getQueueConfigurations()) {
+   private void deployAddressesFromConfiguration() throws Exception {
+      for (CoreAddressConfiguration config : configuration.getAddressConfigurations()) {
+         AddressInfo info = new AddressInfo(SimpleString.toSimpleString(config.getName()));
+         info.setRoutingType(config.getRoutingType());
+         info.setDefaultDeleteOnNoConsumers(config.getDefaultDeleteOnNoConsumers());
+         info.setDefaultMaxConsumers(config.getDefaultMaxConsumers());
+
+         addAddressInfo(info);
+         deployQueuesFromListCoreQueueConfiguration(config.getQueueConfigurations());
+      }
+   }
+
+   private void deployQueuesFromListCoreQueueConfiguration(List<CoreQueueConfiguration> queues) throws Exception {
+      for (CoreQueueConfiguration config : queues) {
          deployQueue(SimpleString.toSimpleString(config.getAddress()), SimpleString.toSimpleString(config.getName()), SimpleString.toSimpleString(config.getFilterString()), config.isDurable(), false);
       }
    }
+   private void deployQueuesFromConfiguration() throws Exception {
+      deployQueuesFromListCoreQueueConfiguration(configuration.getQueueConfigurations());
+   }
 
    private void checkForPotentialOOMEInAddressConfiguration() {
       long totalMaxSizeBytes = 0;
@@ -2173,7 +2192,22 @@ public class ActiveMQServerImpl implements ActiveMQServer {
       }
    }
 
-   private Queue createQueue(final SimpleString address,
+   @Override
+   public AddressInfo addAddressInfo(AddressInfo addressInfo) {
+      return postOffice.addAddressInfo(addressInfo);
+   }
+
+   @Override
+   public AddressInfo removeAddressInfo(SimpleString address) {
+      return postOffice.removeAddressInfo(address);
+   }
+
+   @Override
+   public AddressInfo getAddressInfo(SimpleString address) {
+      return postOffice.removeAddressInfo(address);
+   }
+
+   private Queue createQueue(final SimpleString addressName,
                              final SimpleString queueName,
                              final SimpleString filterString,
                              final SimpleString user,
@@ -2182,6 +2216,7 @@ public class ActiveMQServerImpl implements ActiveMQServer {
                              final boolean ignoreIfExists,
                              final boolean transientQueue,
                              final boolean autoCreated) throws Exception {
+
       final QueueBinding binding = (QueueBinding) postOffice.getBinding(queueName);
       if (binding != null) {
          if (ignoreIfExists) {
@@ -2197,14 +2232,16 @@ public class ActiveMQServerImpl implements ActiveMQServer {
       final long queueID = storageManager.generateID();
 
       final QueueConfig.Builder queueConfigBuilder;
-      if (address == null) {
+      if (addressName == null) {
          queueConfigBuilder = QueueConfig.builderWith(queueID, queueName);
       } else {
-         queueConfigBuilder = QueueConfig.builderWith(queueID, queueName, address);
-
+         queueConfigBuilder = QueueConfig.builderWith(queueID, queueName, addressName);
       }
       final QueueConfig queueConfig = queueConfigBuilder.filter(filter).pagingManager(pagingManager).user(user).durable(durable).temporary(temporary).autoCreated(autoCreated).build();
       final Queue queue = queueFactory.createQueueWith(queueConfig);
+
+      addAddressInfo(new AddressInfo(queue.getAddress()));
+
       if (transientQueue) {
          queue.setConsumersRefCount(new TransientQueueManagerImpl(this, queue.getName()));
       } else if (queue.isAutoCreated()) {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AddressInfo.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AddressInfo.java b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AddressInfo.java
new file mode 100644
index 0000000..03c3fa0
--- /dev/null
+++ b/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/AddressInfo.java
@@ -0,0 +1,67 @@
+/*
+ * 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.server.impl;
+
+import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
+import org.apache.activemq.artemis.api.core.SimpleString;
+
+public class AddressInfo {
+
+   public enum RoutingType {
+      MULTICAST, ANYCAST
+   }
+
+   private final SimpleString name;
+
+   private RoutingType routingType = RoutingType.MULTICAST;
+
+   private boolean defaultDeleteOnNoConsumers;
+
+   private int defaultMaxConsumers;
+
+   public AddressInfo(SimpleString name) {
+      this.name = name;
+   }
+
+   public RoutingType getRoutingType() {
+      return routingType;
+   }
+
+   public void setRoutingType(RoutingType routingType) {
+      this.routingType = routingType;
+   }
+
+   public boolean isDefaultDeleteOnNoConsumers() {
+      return defaultDeleteOnNoConsumers;
+   }
+
+   public void setDefaultDeleteOnNoConsumers(boolean defaultDeleteOnNoConsumers) {
+      this.defaultDeleteOnNoConsumers = defaultDeleteOnNoConsumers;
+   }
+
+   public int getDefaultMaxConsumers() {
+      return defaultMaxConsumers;
+   }
+
+   public void setDefaultMaxConsumers(int defaultMaxConsumers) {
+      this.defaultMaxConsumers = defaultMaxConsumers;
+   }
+
+   public SimpleString getName() {
+      return name;
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/artemis-server/src/test/java/org/apache/activemq/artemis/core/config/impl/FileConfigurationTest.java
----------------------------------------------------------------------
diff --git a/artemis-server/src/test/java/org/apache/activemq/artemis/core/config/impl/FileConfigurationTest.java b/artemis-server/src/test/java/org/apache/activemq/artemis/core/config/impl/FileConfigurationTest.java
index d004a22..ce924c0 100644
--- a/artemis-server/src/test/java/org/apache/activemq/artemis/core/config/impl/FileConfigurationTest.java
+++ b/artemis-server/src/test/java/org/apache/activemq/artemis/core/config/impl/FileConfigurationTest.java
@@ -46,13 +46,14 @@ import org.apache.activemq.artemis.core.security.Role;
 import org.apache.activemq.artemis.core.server.JournalType;
 import org.apache.activemq.artemis.core.server.SecuritySettingPlugin;
 import org.apache.activemq.artemis.core.server.cluster.impl.MessageLoadBalancingType;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.server.impl.LegacyLDAPSecuritySettingPlugin;
 import org.apache.activemq.artemis.core.settings.impl.SlowConsumerPolicy;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.apache.activemq.artemis.core.config.CoreAddressConfiguration.RoutingType.ANYCAST;
-import static org.apache.activemq.artemis.core.config.CoreAddressConfiguration.RoutingType.MULTICAST;
+import static org.apache.activemq.artemis.core.server.impl.AddressInfo.RoutingType.ANYCAST;
+import static org.apache.activemq.artemis.core.server.impl.AddressInfo.RoutingType.MULTICAST;
 
 public class FileConfigurationTest extends ConfigurationImplTest {
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/addressing/AddressingTest.java
----------------------------------------------------------------------
diff --git a/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/addressing/AddressingTest.java b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/addressing/AddressingTest.java
new file mode 100644
index 0000000..43d6071
--- /dev/null
+++ b/tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/addressing/AddressingTest.java
@@ -0,0 +1,21 @@
+/*
+ * 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.tests.integration.addressing;
+
+public class AddressingTest {
+
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/54ad2bc7/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/server/impl/fakes/FakePostOffice.java
----------------------------------------------------------------------
diff --git a/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/server/impl/fakes/FakePostOffice.java b/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/server/impl/fakes/FakePostOffice.java
index 6f23e34..9424fc3 100644
--- a/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/server/impl/fakes/FakePostOffice.java
+++ b/tests/unit-tests/src/test/java/org/apache/activemq/artemis/tests/unit/core/server/impl/fakes/FakePostOffice.java
@@ -33,6 +33,7 @@ import org.apache.activemq.artemis.core.server.Queue;
 import org.apache.activemq.artemis.core.server.QueueCreator;
 import org.apache.activemq.artemis.core.server.RoutingContext;
 import org.apache.activemq.artemis.core.server.ServerMessage;
+import org.apache.activemq.artemis.core.server.impl.AddressInfo;
 import org.apache.activemq.artemis.core.server.impl.MessageReferenceImpl;
 import org.apache.activemq.artemis.core.transaction.Transaction;
 
@@ -60,6 +61,22 @@ public class FakePostOffice implements PostOffice {
    }
 
    @Override
+   public AddressInfo addAddressInfo(AddressInfo addressInfo) {
+      return null;
+   }
+
+
+   @Override
+   public AddressInfo removeAddressInfo(SimpleString address) {
+      return null;
+   }
+
+   @Override
+   public AddressInfo getAddressInfo(SimpleString addressName) {
+      return null;
+   }
+
+   @Override
    public void addBinding(final Binding binding) throws Exception {
 
    }