You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ch...@apache.org on 2009/06/19 16:54:27 UTC

svn commit: r786526 - in /activemq/sandbox/activemq-flow: activemq-broker/src/main/java/org/apache/activemq/apollo/broker/ activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/ activemq-broker/src/main/java/org/apache/activemq/apollo/br...

Author: chirino
Date: Fri Jun 19 14:54:26 2009
New Revision: 786526

URL: http://svn.apache.org/viewvc?rev=786526&view=rev
Log:
- Decoupled the DestinationMap so that it just works on AsciiBuffer 'path' keys instead of Destination obects.  Should make this more re-usable.
- Integrated into the Router package and the Domain classes were not needed anymore.


Added:
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/
      - copied from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/AnyChildPathNode.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/AnyChildDestinationNode.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathFilter.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMap.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMap.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapEntry.java
      - copied, changed from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapEntry.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapNode.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapNode.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathNode.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationNode.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathSupport.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationPath.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PrefixPathFilter.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/PrefixDestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/SimplePathFilter.java
      - copied, changed from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/SimpleDestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/WildcardPathFilter.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/WildcardDestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/
      - copied from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/
    activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapMemoryTest.java
      - copied, changed from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapMemoryTest.java
    activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapTest.java
      - copied, changed from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapTest.java
Removed:
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/Domain.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/QueueDomain.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/TopicDomain.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/AnyChildDestinationNode.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/CompositeDestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/DestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/DestinationMap.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/DestinationMapEntry.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/DestinationMapNode.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/DestinationNode.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/DestinationPath.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PrefixDestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/SimpleDestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/WildcardDestinationFilter.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/
    activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/DestinationMapMemoryTest.java
    activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/DestinationMapTempDestinationTest.java
    activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/DestinationMapTest.java
    activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/
Modified:
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/Router.java
    activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/VirtualHost.java
    activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/ConduitBridge.java
    activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DemandForwardingBridgeSupport.java
    activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DurableConduitBridge.java

Modified: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/Router.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/Router.java?rev=786526&r1=786525&r2=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/Router.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/Router.java Fri Jun 19 14:54:26 2009
@@ -22,12 +22,7 @@
 import java.util.HashMap;
 import java.util.HashSet;
 
-import org.apache.activemq.apollo.broker.DeliveryTarget;
-import org.apache.activemq.apollo.broker.Destination;
-import org.apache.activemq.apollo.broker.Domain;
-import org.apache.activemq.apollo.broker.MessageDelivery;
-import org.apache.activemq.apollo.broker.QueueDomain;
-import org.apache.activemq.apollo.broker.TopicDomain;
+import org.apache.activemq.apollo.broker.path.PathMap;
 import org.apache.activemq.flow.ISourceController;
 import org.apache.activemq.protobuf.AsciiBuffer;
 import org.apache.commons.logging.Log;
@@ -41,34 +36,36 @@
     public static final AsciiBuffer TEMP_TOPIC_DOMAIN = new AsciiBuffer("temp-topic");
     public static final AsciiBuffer TEMP_QUEUE_DOMAIN = new AsciiBuffer("temp-queue");
 
-    private final HashMap<AsciiBuffer, Domain> domains = new HashMap<AsciiBuffer, Domain>();
+    private final HashMap<AsciiBuffer, PathMap<DeliveryTarget>> domains = new HashMap<AsciiBuffer, PathMap<DeliveryTarget>>();
+    
     private VirtualHost virtualHost;
     private BrokerDatabase database;
+    
 
     public Router() {
-        domains.put(QUEUE_DOMAIN, new QueueDomain());
-        domains.put(TOPIC_DOMAIN, new TopicDomain());
-        domains.put(TEMP_QUEUE_DOMAIN, new QueueDomain());
-        domains.put(TEMP_TOPIC_DOMAIN, new TopicDomain());
+        domains.put(QUEUE_DOMAIN, new PathMap<DeliveryTarget>());
+        domains.put(TOPIC_DOMAIN, new PathMap<DeliveryTarget>());
+        domains.put(TEMP_QUEUE_DOMAIN, new PathMap<DeliveryTarget>());
+        domains.put(TEMP_TOPIC_DOMAIN, new PathMap<DeliveryTarget>());
     }
 
-    public Domain getDomain(AsciiBuffer name) {
+    public PathMap<DeliveryTarget> getDomain(AsciiBuffer name) {
         return domains.get(name);
     }
 
-    public Domain putDomain(AsciiBuffer name, Domain domain) {
+    public PathMap<DeliveryTarget> putDomain(AsciiBuffer name, PathMap<DeliveryTarget> domain) {
         return domains.put(name, domain);
     }
 
-    public Domain removeDomain(Object name) {
+    public PathMap<DeliveryTarget> removeDomain(AsciiBuffer name) {
         return domains.remove(name);
     }
 
     public synchronized void bind(Destination destination, DeliveryTarget target) {
         Collection<Destination> destinationList = destination.getDestinations();
         if (destinationList == null) {
-            Domain domain = domains.get(destination.getDomain());
-            domain.bind(destination.getName(), target);
+        	PathMap<DeliveryTarget> domain = domains.get(destination.getDomain());
+            domain.put(destination.getName(), target);
         } else {
             for (Destination d : destinationList) {
                 bind(d, target);
@@ -79,8 +76,8 @@
     public synchronized void unbind(Destination destination, DeliveryTarget target) {
         Collection<Destination> destinationList = destination.getDestinations();
         if (destinationList == null) {
-            Domain domain = domains.get(destination.getDomain());
-            domain.unbind(destination.getName(), target);
+        	PathMap<DeliveryTarget> domain = domains.get(destination.getDomain());
+            domain.remove(destination.getName(), target);
         } else {
             for (Destination d : destinationList) {
                 unbind(d, target);
@@ -124,10 +121,10 @@
         // Handles routing to composite/multi destinations.
         Collection<Destination> destinationList = destination.getDestinations();
         if (destinationList == null) {
-            Domain domain = domains.get(destination.getDomain());
-            Collection<DeliveryTarget> rc = domain.route(destination.getName(), msg);
+        	PathMap<DeliveryTarget> domain = domains.get(destination.getDomain());
+            Collection<DeliveryTarget> rc = domain.get(destination.getName());
             // We can auto create queues in the queue domain..
-            if(rc==null && autoCreate && destination.getDomain().equals(Router.QUEUE_DOMAIN) ) {
+            if(rc.isEmpty() && autoCreate && destination.getDomain().equals(Router.QUEUE_DOMAIN) ) {
             	try {
 					Queue queue = virtualHost.createQueue(destination);
 					rc = new ArrayList<DeliveryTarget>(1);

Modified: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/VirtualHost.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/VirtualHost.java?rev=786526&r1=786525&r2=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/VirtualHost.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/VirtualHost.java Fri Jun 19 14:54:26 2009
@@ -23,6 +23,7 @@
 
 import org.apache.activemq.Service;
 import org.apache.activemq.apollo.broker.ProtocolHandler.ConsumerContext;
+import org.apache.activemq.apollo.broker.path.PathMap;
 import org.apache.activemq.broker.store.Store;
 import org.apache.activemq.broker.store.StoreFactory;
 import org.apache.activemq.protobuf.AsciiBuffer;
@@ -116,10 +117,10 @@
         // Create Queue instances
         for (IQueue<Long, MessageDelivery> iQueue : queueStore.getSharedQueues()) {
             Queue queue = new Queue(iQueue);
-            Domain domain = router.getDomain(Router.QUEUE_DOMAIN);
+            PathMap<DeliveryTarget> domain = router.getDomain(Router.QUEUE_DOMAIN);
             Destination dest = new Destination.SingleDestination(Router.QUEUE_DOMAIN, iQueue.getDescriptor().getQueueName());
             queue.setDestination(dest);
-            domain.add(dest.getName(), queue);
+            domain.put(dest.getName(), queue);
             queues.put(dest.getName(), queue);
         }
         for (Queue queue : queues.values()) {
@@ -156,8 +157,8 @@
             IQueue<Long, MessageDelivery> iQueue = queueStore.createSharedQueue(dest.getName().toString());
             queue = new Queue(iQueue);
             queue.setDestination(dest);
-            Domain domain = router.getDomain(dest.getDomain());
-            domain.add(dest.getName(), queue);
+            PathMap<DeliveryTarget> domain = router.getDomain(dest.getDomain());
+            domain.put(dest.getName(), queue);
             queues.put(dest.getName(), queue);
         }
         queue.start();

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/AnyChildPathNode.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/AnyChildDestinationNode.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/AnyChildPathNode.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/AnyChildPathNode.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/AnyChildDestinationNode.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/AnyChildDestinationNode.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/AnyChildPathNode.java Fri Jun 19 14:54:26 2009
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -23,49 +23,49 @@
 import org.apache.activemq.protobuf.AsciiBuffer;
 
 /**
- * An implementation of {@link DestinationNode} which navigates all the children of the given node
+ * An implementation of {@link PathNode} which navigates all the children of the given node
  * ignoring the name of the current path (so for navigating using * in a wildcard).
  *
  * @version $Revision: 563921 $
  */
-public class AnyChildDestinationNode<Value> implements DestinationNode<Value> {
-    private DestinationNode<Value> node;
+public class AnyChildPathNode<Value> implements PathNode<Value> {
+    private PathNode<Value> node;
 
-    public AnyChildDestinationNode(DestinationNode<Value> node) {
+    public AnyChildPathNode(PathNode<Value> node) {
         this.node = node;
     }
 
     public void appendMatchingValues(Set<Value> answer, ArrayList<AsciiBuffer> paths, int startIndex) {
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    	for (PathNode<Value> child : getChildNodes()) {
             child.appendMatchingValues(answer, paths, startIndex);
         }
     }
 
 
     public void appendMatchingWildcards(Set<Value> answer, ArrayList<AsciiBuffer> paths, int startIndex) {
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    	for (PathNode<Value> child : getChildNodes()) {
             child.appendMatchingWildcards(answer, paths, startIndex);
         }
     }
 
 
     public void appendDescendantValues(Set<Value> answer) {
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    	for (PathNode<Value> child : getChildNodes()) {
             child.appendDescendantValues(answer);
         }
     }
 
-    public DestinationNode<Value> getChild(AsciiBuffer path) {
-        final Collection<DestinationNode<Value>> list = new ArrayList<DestinationNode<Value>>();
-    	for (DestinationNode<Value> child : getChildNodes()) {
-            DestinationNode<Value> answer = child.getChild(path);
+    public PathNode<Value> getChild(AsciiBuffer path) {
+        final Collection<PathNode<Value>> list = new ArrayList<PathNode<Value>>();
+    	for (PathNode<Value> child : getChildNodes()) {
+            PathNode<Value> answer = child.getChild(path);
             if (answer != null) {
                 list.add(answer);
             }
         }
         if (!list.isEmpty()) {
-            return new AnyChildDestinationNode<Value>(this) {
-                protected Collection<DestinationNode<Value>> getChildNodes() {
+            return new AnyChildPathNode<Value>(this) {
+                protected Collection<PathNode<Value>> getChildNodes() {
                     return list;
                 }
             };
@@ -75,7 +75,7 @@
 
     public Collection<Value> getDesendentValues() {
         Collection<Value> answer = new ArrayList<Value>();
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    	for (PathNode<Value> child : getChildNodes()) {
             answer.addAll(child.getDesendentValues());
         }
         return answer;
@@ -83,16 +83,16 @@
 
     public Collection<Value> getValues() {
         Collection<Value> answer = new ArrayList<Value>();
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    	for (PathNode<Value> child : getChildNodes()) {
             answer.addAll(child.getValues());
         }
         return answer;
     }
 
 
-    public Collection<DestinationNode<Value>> getChildren() {
-        Collection<DestinationNode<Value>>  answer = new ArrayList<DestinationNode<Value>> ();
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    public Collection<PathNode<Value>> getChildren() {
+        Collection<PathNode<Value>>  answer = new ArrayList<PathNode<Value>> ();
+    	for (PathNode<Value> child : getChildNodes()) {
             answer.addAll(child.getChildren());
         }
         return answer;
@@ -100,7 +100,7 @@
 
     public Collection<Value> removeDesendentValues() {
         Collection<Value> answer = new ArrayList<Value>();
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    	for (PathNode<Value> child : getChildNodes()) {
             answer.addAll(child.removeDesendentValues());
         }
         return answer;
@@ -108,13 +108,13 @@
 
     public Collection<Value> removeValues() {
         Collection<Value> answer = new ArrayList<Value>();
-    	for (DestinationNode<Value> child : getChildNodes()) {
+    	for (PathNode<Value> child : getChildNodes()) {
             answer.addAll(child.removeValues());
         }
         return answer;
     }
 
-    protected Collection<DestinationNode<Value>> getChildNodes() {
+    protected Collection<PathNode<Value>> getChildNodes() {
         return node.getChildren();
     }
 }

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathFilter.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationFilter.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathFilter.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathFilter.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationFilter.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationFilter.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathFilter.java Fri Jun 19 14:54:26 2009
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
-import java.util.Collection;
 
 import org.apache.activemq.apollo.broker.Destination;
 import org.apache.activemq.filter.BooleanExpression;
@@ -28,47 +27,59 @@
 
 
 /**
- * Represents a filter which only operates on Destinations
+ * Represents a filter which only operates on a path
  * 
  * @version $Revision: 1.3 $
  */
-public abstract class DestinationFilter implements BooleanExpression {
+public abstract class PathFilter implements BooleanExpression {
 
     public static final AsciiBuffer ANY_DESCENDENT = new AsciiBuffer(">");
     public static final AsciiBuffer ANY_CHILD = new AsciiBuffer("*");
     
 	public boolean matches(MessageEvaluationContext message) throws FilterException {
 		Destination destination = message.getDestination();
-		return matches(destination);
+		return matches(destination.getName());
 	}
+	
 	public Object evaluate(MessageEvaluationContext message) throws FilterException {
 		return matches(message) ? Boolean.TRUE : Boolean.FALSE;
 	}
 	
-    public abstract boolean matches(Destination destination);
+    public abstract boolean matches(AsciiBuffer path);
 
-    public static DestinationFilter parseFilter(Destination destination) {
-    	Collection<Destination> destinations = destination.getDestinations();
-        if (destinations!=null) {
-            return new CompositeDestinationFilter(destination);
-        }
-        ArrayList<AsciiBuffer> paths = DestinationPath.parse(destination);
+    public static PathFilter parseFilter(AsciiBuffer path) {
+        ArrayList<AsciiBuffer> paths = PathSupport.parse(path);
         int idx = paths.size() - 1;
         if (idx >= 0) {
         	AsciiBuffer lastPath = paths.get(idx);
             if (lastPath.equals(ANY_DESCENDENT)) {
-                return new PrefixDestinationFilter(paths);
+                return new PrefixPathFilter(paths);
             } else {
                 while (idx >= 0) {
                     lastPath = paths.get(idx--);
                     if (lastPath.equals(ANY_CHILD)) {
-                        return new WildcardDestinationFilter(paths);
+                        return new WildcardPathFilter(paths);
                     }
                 }
             }
         }
 
         // if none of the paths contain a wildcard then use equality
-        return new SimpleDestinationFilter(destination);
+        return new SimplePathFilter(path);
     }
+    
+    public static boolean containsWildCards(AsciiBuffer path) {
+    	byte b1 = ANY_DESCENDENT.getData()[0];
+    	byte b2 = ANY_CHILD.getData()[0];
+    	
+    	byte[] data = path.getData();
+    	int length = path.getOffset()+path.getLength();
+		for (int i = path.getOffset(); i < length; i++) {
+			if( data[i] == b1 || data[i]==b2 ) {
+				return true;
+			}
+		}
+		return false;
+    }
+    
 }

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMap.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMap.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMap.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMap.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMap.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMap.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMap.java Fri Jun 19 14:54:26 2009
@@ -14,89 +14,65 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.activemq.apollo.broker.Destination;
 import org.apache.activemq.protobuf.AsciiBuffer;
 
 /**
  * A Map-like data structure allowing values to be indexed by
- * {@link Destination} and retrieved by destination - supporting both *
- * and &gt; style of wildcard as well as composite destinations. <br>
+ * {@link AsciiBuffer} and retrieved by path - supporting both *
+ * and &gt; style of wildcard as well as composite paths. <br>
  * This class assumes that the index changes rarely but that fast lookup into
  * the index is required. So this class maintains a pre-calculated index for
- * destination steps. So looking up the values for "TEST.*" or "*.TEST" will be
+ * path steps. So looking up the values for "TEST.*" or "*.TEST" will be
  * pretty fast. <br>
  * Looking up of a value could return a single value or a List of matching
- * values if a wildcard or composite destination is used.
+ * values if a wildcard or composite path is used.
  * 
  * @version $Revision: 1.3 $
  */
-public class DestinationMap<Value> {
-    protected static final AsciiBuffer ANY_DESCENDENT = DestinationFilter.ANY_DESCENDENT;
-    protected static final AsciiBuffer ANY_CHILD = DestinationFilter.ANY_CHILD;
+public class PathMap<Value> {
+    protected static final AsciiBuffer ANY_DESCENDENT = PathFilter.ANY_DESCENDENT;
+    protected static final AsciiBuffer ANY_CHILD = PathFilter.ANY_CHILD;
 
-    private final DestinationMapNode<Value> root = new DestinationMapNode<Value>(null);
+    private final PathMapNode<Value> root = new PathMapNode<Value>(null);
 
     /**
-     * Looks up the value(s) matching the given Destination key. For simple
-     * destinations this is typically a List of one single value, for wild cards
-     * or composite destinations this will typically be a List of matching
+     * Looks up the value(s) matching the given AsciiBuffer key. For simple
+     * paths this is typically a List of one single value, for wild cards
+     * or composite paths this will typically be a List of matching
      * values.
      * 
-     * @param key the destination to lookup
+     * @param key the path to lookup
      * @return a List of matching values or an empty list if there are no
      *         matching values.
      */
-    public synchronized Set<Value> get(Destination key) {
-    	Collection<Destination> destinations = key.getDestinations();
-        if (destinations!=null) {
-        	HashSet<Value> answer = new HashSet<Value>(destinations.size());
-        	for (Destination childDestination : destinations) {
-                answer.addAll(get(childDestination));
-            }
-            return answer;
-        }
+    public synchronized Set<Value> get(AsciiBuffer key) {
         return findWildcardMatches(key);
     }
 
-    public synchronized void put(Destination key, Value value) {
-    	Collection<Destination> destinations = key.getDestinations();
-        if (destinations!=null) {
-        	for (Destination childDestination : destinations) {
-                put(childDestination, value);
-            }
-            return;
-        }
-        ArrayList<AsciiBuffer> paths = DestinationPath.parse(key);
+    public synchronized void put(AsciiBuffer key, Value value) {
+        ArrayList<AsciiBuffer> paths = PathSupport.parse(key);
         root.add(paths, 0, value);
     }
 
     /**
-     * Removes the value from the associated destination
+     * Removes the value from the associated path
      */
-    public synchronized void remove(Destination key, Value value) {
-    	Collection<Destination> destinations = key.getDestinations();
-        if (destinations!=null) {
-        	for (Destination childDestination : destinations) {
-                remove(childDestination, value);
-            }
-            return;
-        }
-        ArrayList<AsciiBuffer> paths = DestinationPath.parse(key);
+    public synchronized void remove(AsciiBuffer key, Value value) {
+        ArrayList<AsciiBuffer> paths = PathSupport.parse(key);
         root.remove(paths, 0, value);
 
     }
 
-    public DestinationMapNode<Value> getRootNode() {
+    public PathMapNode<Value> getRootNode() {
         return root;
     }
 
@@ -104,18 +80,18 @@
     // -------------------------------------------------------------------------
 
     /**
-     * A helper method to allow the destination map to be populated from a
+     * A helper method to allow the path map to be populated from a
      * dependency injection framework such as Spring
      */
     @SuppressWarnings("unchecked")
-	protected void setEntries(List<DestinationMapEntry> entries) {
-    	for (DestinationMapEntry entry : entries) {
-            put(entry.getDestination(), (Value) entry);
+	protected void setEntries(List<PathMapEntry> entries) {
+    	for (PathMapEntry entry : entries) {
+            put(entry.getKey(), (Value) entry);
         }
     }
 
-    protected Set<Value> findWildcardMatches(Destination key) {
-    	ArrayList<AsciiBuffer> paths = DestinationPath.parse(key);
+    protected Set<Value> findWildcardMatches(AsciiBuffer key) {
+    	ArrayList<AsciiBuffer> paths = PathSupport.parse(key);
         HashSet<Value> answer = new HashSet<Value>();
         root.appendMatchingValues(answer, paths, 0);
         return answer;
@@ -125,30 +101,23 @@
      * @param key
      * @return
      */
-    public Set<Value> removeAll(Destination key) {
+    public Set<Value> removeAll(AsciiBuffer key) {
     	HashSet<Value> rc = new HashSet<Value>();
-    	Collection<Destination> destinations = key.getDestinations();
-        if (destinations!=null) {
-        	for (Destination childDestination : destinations) {
-                rc.addAll(removeAll(childDestination));
-            }
-            return rc;
-        }
-        ArrayList<AsciiBuffer> paths = DestinationPath.parse(key);
+        ArrayList<AsciiBuffer> paths = PathSupport.parse(key);
         root.removeAll(rc, paths, 0);
         return rc;
     }
 
     /**
-     * Returns the value which matches the given destination or null if there is
+     * Returns the value which matches the given path or null if there is
      * no matching value. If there are multiple values, the results are sorted
      * and the last item (the biggest) is returned.
      * 
-     * @param destination the destination to find the value for
+     * @param path the path to find the value for
      * @return the largest matching value or null if no value matches
      */
-    public Value chooseValue(Destination destination) {
-        Set<Value> set = get(destination);
+    public Value chooseValue(AsciiBuffer path) {
+        Set<Value> set = get(path);
         if (set == null || set.isEmpty()) {
             return null;
         }

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapEntry.java (from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapEntry.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapEntry.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapEntry.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapEntry.java&r1=786461&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapEntry.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapEntry.java Fri Jun 19 14:54:26 2009
@@ -14,71 +14,45 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
-import org.apache.activemq.apollo.broker.Destination;
-import org.springframework.beans.factory.InitializingBean;
+import org.apache.activemq.protobuf.AsciiBuffer;
 
 /**
- * A base class for entry objects used to construct a destination based policy
+ * A base class for entry objects used to construct a path based policy
  * map.
  * 
  * @version $Revision: 1.1 $
  */
-public abstract class DestinationMapEntry implements InitializingBean, Comparable<DestinationMapEntry> {
+public abstract class PathMapEntry implements Comparable<PathMapEntry> {
 
-    private Destination destination;
+    private AsciiBuffer key;
 
-    public int compareTo(DestinationMapEntry that) {
+    public int compareTo(PathMapEntry that) {
     	if( that == null )
     		return 1;
-        return compare(destination, that.destination);
+        return compare(key, that.key);
     }
     
-    public static int compare(Destination destination, Destination destination2) {
-        if (destination == destination2) {
+    public static int compare(AsciiBuffer path, AsciiBuffer path2) {
+        if (path == path2) {
             return 0;
         }
-        if (destination == null) {
+        if (path == null) {
             return -1;
-        } else if (destination2 == null) {
+        } else if (path2 == null) {
             return 1;
         } else {
-        	int rc = destination.getDomain().compareTo(destination2.getDomain());
-        	if( rc == 0 ) {
-        		rc = destination.getName().compareTo(destination2.getName());;
-        	}
-        	return rc;
+       		return path.compareTo(path2);
         }
     }
     
-
-//    /**
-//     * A helper method to set the destination from a configuration file
-//     */
-//    public void setQueue(String name) {
-//        setDestination(new ActiveMQQueue(name));
-//    }
-//
-//    /**
-//     * A helper method to set the destination from a configuration file
-//     */
-//    public void setTopic(String name) {
-//        setDestination(new ActiveMQTopic(name));
-//    }
-
-    public Destination getDestination() {
-        return destination;
+    public AsciiBuffer getKey() {
+        return key;
     }
 
-    public void setDestination(Destination destination) {
-        this.destination = destination;
-    }
-
-    public void afterPropertiesSet() throws Exception {
-        if (destination == null) {
-            throw new IllegalArgumentException("You must specify the 'destination' property");
-        }
+    public void setKey(AsciiBuffer key) {
+        this.key = key;
     }
 
 }

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapNode.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapNode.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapNode.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapNode.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapNode.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapNode.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathMapNode.java Fri Jun 19 14:54:26 2009
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -27,23 +27,22 @@
 import org.apache.activemq.protobuf.AsciiBuffer;
 
 /**
- * An implementation class used to implement {@link DestinationMap}
+ * An implementation class used to implement {@link PathMap}
  * 
  * @version $Revision: 1.2 $
  */
-public class DestinationMapNode<Value> implements DestinationNode<Value> {
-    protected static final AsciiBuffer ANY_CHILD = DestinationMap.ANY_CHILD;
-    protected static final AsciiBuffer ANY_DESCENDENT = DestinationMap.ANY_DESCENDENT;
+public class PathMapNode<Value> implements PathNode<Value> {
+    protected static final AsciiBuffer ANY_CHILD = PathMap.ANY_CHILD;
+    protected static final AsciiBuffer ANY_DESCENDENT = PathMap.ANY_DESCENDENT;
 
-    // we synchornize at the DestinationMap level
-    private DestinationMapNode<Value> parent;
+    // we synchronize at the PathMap level
+    private PathMapNode<Value> parent;
     private List<Value> values = new ArrayList<Value>();
-    private Map<AsciiBuffer, DestinationNode<Value>> childNodes = new HashMap<AsciiBuffer, DestinationNode<Value>>();
+    private Map<AsciiBuffer, PathNode<Value>> childNodes = new HashMap<AsciiBuffer, PathNode<Value>>();
     private AsciiBuffer path = new AsciiBuffer("Root");
-    // private DestinationMapNode anyChild;
     private int pathLength;
 
-    public DestinationMapNode(DestinationMapNode<Value> parent) {
+    public PathMapNode(PathMapNode<Value> parent) {
         this.parent = parent;
         if (parent == null) {
             pathLength = 0;
@@ -56,14 +55,14 @@
      * Returns the child node for the given named path or null if it does not
      * exist
      */
-    public DestinationMapNode<Value> getChild(AsciiBuffer path) {
-        return (DestinationMapNode<Value>)childNodes.get(path);
+    public PathMapNode<Value> getChild(AsciiBuffer path) {
+        return (PathMapNode<Value>)childNodes.get(path);
     }
 
     /**
      * Returns the child nodes
      */
-    public Collection<DestinationNode<Value>> getChildren() {
+    public Collection<PathNode<Value>> getChildren() {
         return childNodes.values();
     }
 
@@ -75,8 +74,8 @@
      * Returns the child node for the given named path, lazily creating one if
      * it does not yet exist
      */
-    public DestinationMapNode<Value> getChildOrCreate(AsciiBuffer asciiBuffer) {
-        DestinationMapNode<Value> answer = (DestinationMapNode<Value>)childNodes.get(asciiBuffer);
+    public PathMapNode<Value> getChildOrCreate(AsciiBuffer asciiBuffer) {
+        PathMapNode<Value> answer = (PathMapNode<Value>)childNodes.get(asciiBuffer);
         if (answer == null) {
             answer = createChildNode();
             answer.path = asciiBuffer;
@@ -86,15 +85,6 @@
     }
 
     /**
-     * Returns the node which represents all children (i.e. the * node)
-     */
-    // public DestinationMapNode getAnyChildNode() {
-    // if (anyChild == null) {
-    // anyChild = createChildNode();
-    // }
-    // return anyChild;
-    // }
-    /**
      * Returns a mutable List of the values available at this node in the tree
      */
     public List<Value> getValues() {
@@ -164,7 +154,7 @@
     }
 
     public void removeAll(Set<Value> answer, ArrayList<AsciiBuffer> paths, int startIndex) {
-        DestinationNode<Value> node = this;
+        PathNode<Value> node = this;
         int size = paths.size();
         for (int i = startIndex; i < size && node != null; i++) {
 
@@ -177,7 +167,7 @@
             node.appendMatchingWildcards(answer, paths, i);
             if (path.equals(ANY_CHILD)) {
                 // node = node.getAnyChildNode();
-                node = new AnyChildDestinationNode<Value>(node);
+                node = new AnyChildPathNode<Value>(node);
             } else {
                 node = node.getChild(path);
             }
@@ -193,7 +183,7 @@
         answer.addAll(values);
 
         // lets add all the children too
-        for (DestinationNode<Value> child : childNodes.values()) {
+        for (PathNode<Value> child : childNodes.values()) {
 			child.appendDescendantValues(answer);
         }
 
@@ -206,8 +196,8 @@
     /**
      * Factory method to create a child node
      */
-    protected DestinationMapNode<Value> createChildNode() {
-        return new DestinationMapNode<Value>(this);
+    protected PathMapNode<Value> createChildNode() {
+        return new PathMapNode<Value>(this);
     }
 
     /**
@@ -217,7 +207,7 @@
         if (idx - 1 > pathLength) {
             return;
         }
-        DestinationMapNode<Value> wildCardNode = getChild(ANY_CHILD);
+        PathMapNode<Value> wildCardNode = getChild(ANY_CHILD);
         if (wildCardNode != null) {
             wildCardNode.appendMatchingValues(answer, paths, idx + 1);
         }
@@ -228,7 +218,7 @@
     }
 
     public void appendMatchingValues(Set<Value> answer, ArrayList<AsciiBuffer> paths, int startIndex) {
-        DestinationNode<Value> node = this;
+        PathNode<Value> node = this;
         boolean couldMatchAny = true;
         int size = paths.size();
         for (int i = startIndex; i < size && node != null; i++) {
@@ -242,7 +232,7 @@
             node.appendMatchingWildcards(answer, paths, i);
 
             if (path.equals(ANY_CHILD)) {
-                node = new AnyChildDestinationNode<Value>(node);
+                node = new AnyChildPathNode<Value>(node);
             } else {
                 node = node.getChild(path);
             }
@@ -251,7 +241,7 @@
             answer.addAll(node.getValues());
             if (couldMatchAny) {
                 // lets allow FOO.BAR to match the FOO.BAR.> entry in the map
-                DestinationNode<Value> child = node.getChild(ANY_DESCENDENT);
+                PathNode<Value> child = node.getChild(ANY_DESCENDENT);
                 if (child != null) {
                     answer.addAll(child.getValues());
                 }
@@ -269,7 +259,7 @@
         }
     }
 
-    protected void removeChild(DestinationMapNode<Value> node) {
+    protected void removeChild(PathMapNode<Value> node) {
         childNodes.remove(node.getPath());
         pruneIfEmpty();
     }

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathNode.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationNode.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathNode.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathNode.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationNode.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationNode.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathNode.java Fri Jun 19 14:54:26 2009
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -23,11 +23,11 @@
 import org.apache.activemq.protobuf.AsciiBuffer;
 
 /**
- * Represents a node in the {@link DestinationMap} tree
+ * Represents a node in the {@link PathMap} tree
  *
  * @version $Revision: 563921 $
  */
-public interface DestinationNode<Value> {
+public interface PathNode<Value> {
     void appendMatchingValues(Set<Value> answer, ArrayList<AsciiBuffer> paths, int startIndex);
 
     void appendMatchingWildcards(Set<Value> answer, ArrayList<AsciiBuffer> paths, int startIndex);
@@ -36,11 +36,11 @@
 
     Collection<Value> getDesendentValues();
 
-    DestinationNode<Value> getChild(AsciiBuffer path);
+    PathNode<Value> getChild(AsciiBuffer path);
 
     Collection<Value> getValues();
 
-    Collection<DestinationNode<Value>> getChildren();
+    Collection<PathNode<Value>> getChildren();
 
     Collection<Value> removeDesendentValues();
 

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathSupport.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationPath.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathSupport.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathSupport.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationPath.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/DestinationPath.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PathSupport.java Fri Jun 19 14:54:26 2009
@@ -15,27 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
 
-import org.apache.activemq.apollo.broker.Destination;
 import org.apache.activemq.protobuf.AsciiBuffer;
 
 /**
- * Helper class for decomposing a Destination into a number of paths
+ * Helper class for decomposing a AsciiBuffer based path into it path component parts.
  * 
  * @version $Revision: 1.3 $
  */
-public final class DestinationPath {
+public final class PathSupport {
     protected static final byte SEPARATOR = '.';
 
-    private DestinationPath() {    
+    private PathSupport() {    
     }
     
-    public static ArrayList<AsciiBuffer> parse(AsciiBuffer domain, AsciiBuffer subject) {
+    public static ArrayList<AsciiBuffer> parse(AsciiBuffer subject) {
     	ArrayList<AsciiBuffer> list = new ArrayList<AsciiBuffer>(10);
-        list.add(domain);
         int previous = 0;
         int lastIndex = subject.getLength() - 1;
         while (true) {
@@ -52,10 +50,6 @@
         return list;
     }
 
-    public static ArrayList<AsciiBuffer> parse(Destination destination) {
-        return parse(destination.getDomain(), destination.getName());
-    }
-
     /**
      * Converts the paths to a single String seperated by dots.
      * 

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PrefixPathFilter.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/PrefixDestinationFilter.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PrefixPathFilter.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PrefixPathFilter.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/PrefixDestinationFilter.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/PrefixDestinationFilter.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/PrefixPathFilter.java Fri Jun 19 14:54:26 2009
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
 
-import org.apache.activemq.apollo.broker.Destination;
 import org.apache.activemq.protobuf.AsciiBuffer;
 
 
@@ -28,7 +27,7 @@
  *
  * @version $Revision: 1.2 $
  */
-public class PrefixDestinationFilter extends DestinationFilter {
+public class PrefixPathFilter extends PathFilter {
 
     private ArrayList<AsciiBuffer> prefixes;
 
@@ -37,17 +36,17 @@
      *
      * @param paths
      */
-    public PrefixDestinationFilter(ArrayList<AsciiBuffer> paths) {
+    public PrefixPathFilter(ArrayList<AsciiBuffer> paths) {
         this.prefixes = paths;
     }
 
-    public boolean matches(Destination destination) {
-        ArrayList<AsciiBuffer> path = DestinationPath.parse(destination);
+    public boolean matches(AsciiBuffer path) {
+        ArrayList<AsciiBuffer> parts = PathSupport.parse(path);
         int length = prefixes.size();
-        if (path.size() >= length) {
+        if (parts.size() >= length) {
             int size = length - 1;
             for (int i = 0; i < size; i++) {
-                if (!prefixes.get(i).equals(path.get(i))) {
+                if (!prefixes.get(i).equals(parts.get(i))) {
                     return false;
                 }
             }
@@ -57,11 +56,11 @@
     }
 
     public String getText() {
-        return DestinationPath.toString(prefixes);
+        return PathSupport.toString(prefixes);
     }
 
     public String toString() {
-        return super.toString() + "[destination: " + getText() + "]";
+        return super.toString() + "[path: " + getText() + "]";
     }
 
     public boolean isWildcard() {

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/SimplePathFilter.java (from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/SimpleDestinationFilter.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/SimplePathFilter.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/SimplePathFilter.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/SimpleDestinationFilter.java&r1=786461&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/SimpleDestinationFilter.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/SimplePathFilter.java Fri Jun 19 14:54:26 2009
@@ -15,29 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.activemq.apollo.broker.wildcard;
-
-import org.apache.activemq.apollo.broker.Destination;
+package org.apache.activemq.apollo.broker.path;
 
+import org.apache.activemq.protobuf.AsciiBuffer;
 
 /**
- * Matches messages sent to an exact destination
- *
+ * Matches messages sent to an exact path
+ * 
  * @version $Revision: 1.3 $
  */
-public class SimpleDestinationFilter extends DestinationFilter {
+public class SimplePathFilter extends PathFilter {
 
-    private Destination destination;
+	private AsciiBuffer path;
 
-    public SimpleDestinationFilter(Destination destination) {
-        this.destination = destination;
-    }
+	public SimplePathFilter(AsciiBuffer path) {
+		this.path = path;
+	}
 
-    public boolean matches(Destination destination) {
-        return this.destination.equals(destination);
-    }
+	public boolean matches(AsciiBuffer path) {
+		return this.path.equals(path);
+	}
 
-    public boolean isWildcard() {
-        return false;
-    }
+	public boolean isWildcard() {
+		return false;
+	}
 }

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/WildcardPathFilter.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/WildcardDestinationFilter.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/WildcardPathFilter.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/WildcardPathFilter.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/WildcardDestinationFilter.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/wildcard/WildcardDestinationFilter.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/main/java/org/apache/activemq/apollo/broker/path/WildcardPathFilter.java Fri Jun 19 14:54:26 2009
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
 
-import org.apache.activemq.apollo.broker.Destination;
 import org.apache.activemq.protobuf.AsciiBuffer;
 
 
@@ -29,7 +28,7 @@
  *
  * @version $Revision: 1.2 $
  */
-public class WildcardDestinationFilter extends DestinationFilter {
+public class WildcardPathFilter extends PathFilter {
 
     private AsciiBuffer[] prefixes;
 
@@ -38,23 +37,23 @@
      *
      * @param paths
      */
-    public WildcardDestinationFilter(ArrayList<AsciiBuffer> paths) {
+    public WildcardPathFilter(ArrayList<AsciiBuffer> paths) {
         this.prefixes = new AsciiBuffer[paths.size()];
         for (int i = 0; i < paths.size(); i++) {
         	AsciiBuffer prefix = paths.get(i);
-            if (!prefix.equals(DestinationFilter.ANY_CHILD)) {
+            if (!prefix.equals(PathFilter.ANY_CHILD)) {
                 this.prefixes[i] = prefix;
             }
         }
     }
 
-    public boolean matches(Destination destination) {
-        ArrayList<AsciiBuffer> path = DestinationPath.parse(destination);
+    public boolean matches(AsciiBuffer path) {
+        ArrayList<AsciiBuffer> parts = PathSupport.parse(path);
         int length = prefixes.length;
-        if (path.size() == length) {
+        if (parts.size() == length) {
             for (int i = 0; i < length; i++) {
             	AsciiBuffer prefix = prefixes[i];
-                if (prefix != null && !prefix.equals(path.get(i))) {
+                if (prefix != null && !prefix.equals(parts.get(i))) {
                     return false;
                 }
             }
@@ -67,11 +66,11 @@
     public String getText() {
     	ArrayList<AsciiBuffer> t = new ArrayList<AsciiBuffer>(prefixes.length);
     	t.toArray(prefixes);
-        return DestinationPath.toString(t);
+        return PathSupport.toString(t);
     }
 
     public String toString() {
-        return super.toString() + "[destination: " + getText() + "]";
+        return super.toString() + "[path: " + getText() + "]";
     }
 
     public boolean isWildcard() {

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapMemoryTest.java (from r786461, activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapMemoryTest.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapMemoryTest.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapMemoryTest.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapMemoryTest.java&r1=786461&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapMemoryTest.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapMemoryTest.java Fri Jun 19 14:54:26 2009
@@ -14,24 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
+
+import java.util.Set;
 
 import junit.framework.TestCase;
 
-import org.apache.activemq.apollo.broker.Destination;
-import org.apache.activemq.apollo.broker.Router;
-import org.apache.activemq.apollo.broker.wildcard.DestinationMap;
+import org.apache.activemq.apollo.broker.path.PathMap;
 import org.apache.activemq.protobuf.AsciiBuffer;
 
-public class DestinationMapMemoryTest extends TestCase {
+public class PathMapMemoryTest extends TestCase {
 
-    public void testLongDestinationPath() throws Exception {
-    	Destination d1 = new Destination.SingleDestination(Router.TOPIC_DOMAIN, new AsciiBuffer("1.2.3.4.5.6.7.8.9.10.11.12.13.14.15.16.17.18"));
-        DestinationMap<String> map = new DestinationMap<String>();
+    public void testLongPath() throws Exception {
+    	AsciiBuffer d1 = new AsciiBuffer("1.2.3.4.5.6.7.8.9.10.11.12.13.14.15.16.17.18");
+        PathMap<String> map = new PathMap<String>();
         map.put(d1, "test");
     }
 
-    public void testVeryLongestinationPaths() throws Exception {
+    public void testVeryLongPaths() throws Exception {
 
         for (int i = 1; i < 100; i++) {
             String name = "1";
@@ -40,13 +40,29 @@
             }
             // System.out.println("Checking: " + name);
             try {
-            	Destination d1 = new Destination.SingleDestination(Router.TOPIC_DOMAIN, new AsciiBuffer(name));
-                DestinationMap<String> map = new DestinationMap<String>();
+            	AsciiBuffer d1 = new AsciiBuffer(name);
+                PathMap<String> map = new PathMap<String>();
                 map.put(d1, "test");
             } catch (Throwable e) {
                 fail("Destination name too long: " + name + " : " + e);
             }
         }
     }
+    
+    public void testLotsOfPaths() throws Exception {
+        PathMap<Object> map = new PathMap<Object>();
+        Object value = new Object();
+        int count = 1000;
+        for (int i = 0; i < count; i++) {
+            AsciiBuffer queue = new AsciiBuffer("connection:"+i);
+            map.put(queue, value);
+        }
+        for (int i = 0; i < count; i++) {
+            AsciiBuffer queue = new AsciiBuffer("connection:"+i);
+            map.remove(queue, value);
+            Set<Object> set = map.get(queue);
+            assertTrue(set.isEmpty());
+        }
+    }    
 
 }

Copied: activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapTest.java (from r786503, activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapTest.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapTest.java?p2=activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapTest.java&p1=activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapTest.java&r1=786503&r2=786526&rev=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/wildcard/DestinationMapTest.java (original)
+++ activemq/sandbox/activemq-flow/activemq-broker/src/test/java/org/apache/activemq/apollo/broker/path/PathMapTest.java Fri Jun 19 14:54:26 2009
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.activemq.apollo.broker.wildcard;
+package org.apache.activemq.apollo.broker.path;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -25,18 +25,15 @@
 
 import junit.framework.TestCase;
 
-import org.apache.activemq.apollo.broker.Destination;
-import org.apache.activemq.apollo.broker.Router;
+import org.apache.activemq.apollo.broker.path.PathMap;
 import org.apache.activemq.protobuf.AsciiBuffer;
 
-public class DestinationMapTest extends TestCase {
-    protected DestinationMap<String> map = new DestinationMap<String>();
+public class PathMapTest extends TestCase {
+    protected PathMap<String> map = new PathMap<String>();
 
-    protected Destination d1 = createDestination("TEST.D1");
-    protected Destination d2 = createDestination("TEST.BAR.D2");
-    protected Destination d3 = createDestination("TEST.BAR.D3");
-    protected Destination compositeDestination1 = createDestination("TEST.D1,TEST.BAR.D2");
-    protected Destination compositeDestination2 = createDestination("TEST.D1,TEST.BAR.D3");
+    protected AsciiBuffer d1 = createDestination("TEST.D1");
+    protected AsciiBuffer d2 = createDestination("TEST.BAR.D2");
+    protected AsciiBuffer d3 = createDestination("TEST.BAR.D3");
 
     protected String v1 = "value1";
     protected String v2 = "value2";
@@ -45,27 +42,16 @@
     protected String v5 = "value5";
     protected String v6 = "value6";
 
-    public void testQueueAndTopicWithSameName() throws Exception {
-        Destination q1 = new Destination.SingleDestination(Router.QUEUE_DOMAIN, new AsciiBuffer("foo"));
-        Destination t1 = new Destination.SingleDestination(Router.TOPIC_DOMAIN, new AsciiBuffer("foo"));
-
-        map.put(q1, v1);
-        map.put(t1, v2);
-
-        assertMapValue(q1, v1);
-        assertMapValue(t1, v2);
-    }
-
-    public void testCompositeDestinations() throws Exception {
-        Destination d1 = createDestination("TEST.BAR.D2");
-        Destination d2 = createDestination("TEST.BAR.D3");
+    public void testCompositePaths() throws Exception {
+        AsciiBuffer d1 = createDestination("TEST.BAR.D2");
+        AsciiBuffer d2 = createDestination("TEST.BAR.D3");
         map.put(d1, v1);
         map.put(d2, v2);
         map.get(createDestination("TEST.BAR.D2,TEST.BAR.D3"));
 
     }
 
-    public void testSimpleDestinations() throws Exception {
+    public void testSimplePaths() throws Exception {
         map.put(d1, v1);
         map.put(d2, v2);
         map.put(d3, v3);
@@ -85,28 +71,8 @@
         assertMapValue(d3, null);
     }
 
-    public void testSimpleAndCompositeDestinations() throws Exception {
-        map.put(d1, v1);
-        map.put(compositeDestination1, v2);
-        map.put(compositeDestination2, v3);
-
-        Set<String> set = map.get(d1);
-        System.out.println(set);
-        
-        
-        assertMapValue("TEST.D1", v1, v2, v3);
-        assertMapValue(d2, v2);
-        assertMapValue(d3, v3);
-        assertMapValue(compositeDestination1, v1, v2, v3);
-        assertMapValue(compositeDestination2, v1, v2, v3);
-
-        map.remove(compositeDestination1, v2);
-        map.remove(compositeDestination2, v3);
-
-        assertMapValue("TEST.D1", v1);
-    }
 
-    public void testLookupOneStepWildcardDestinations() throws Exception {
+    public void testLookupOneStepWildcardPaths() throws Exception {
         map.put(d1, v1);
         map.put(d2, v2);
         map.put(d3, v3);
@@ -131,7 +97,7 @@
         assertMapValue("TEST.BAR.*", v2, v3);
     }
 
-    public void testLookupMultiStepWildcardDestinations() throws Exception {
+    public void testLookupMultiStepWildcardPaths() throws Exception {
         map.put(d1, v1);
         map.put(d2, v2);
         map.put(d3, v3);
@@ -347,12 +313,12 @@
     }
 
     protected void remove(String name, String value) {
-        Destination destination = createDestination(name);
+        AsciiBuffer destination = createDestination(name);
         map.remove(destination, value);
     }
 
     protected void assertMapValue(String destinationName, Object expected) {
-        Destination destination = createDestination(destinationName);
+        AsciiBuffer destination = createDestination(destinationName);
         assertMapValue(destination, expected);
     }
 
@@ -364,7 +330,7 @@
         assertMapValue(destinationName, Arrays.asList(new Object[] {expected1, expected2, expected3}));
     }
     
-    protected void assertMapValue(Destination destination, Object expected1, Object expected2, Object expected3) {
+    protected void assertMapValue(AsciiBuffer destination, Object expected1, Object expected2, Object expected3) {
         assertMapValue(destination, Arrays.asList(new Object[] {expected1, expected2, expected3}));
     }
 
@@ -381,7 +347,7 @@
     }
 
     @SuppressWarnings("unchecked")
-    protected void assertMapValue(Destination destination, Object expected) {
+    protected void assertMapValue(AsciiBuffer destination, Object expected) {
         List expectedList = null;
         if (expected == null) {
             expectedList = Collections.EMPTY_LIST;
@@ -398,16 +364,7 @@
         assertEquals("map value for destinationName:  " + destination, expectedList, actual);
     }
 
-    protected Destination createDestination(String name) {
-    	String[] split = name.split(",");
-    	if( split.length == 1 ) {
-    		return new Destination.SingleDestination(Router.QUEUE_DOMAIN, new AsciiBuffer(name));
-    	} else {
-    		Destination.MultiDestination rc =  new Destination.MultiDestination();
-    		for (int i = 0; i < split.length; i++) {
-				rc.add(createDestination(split[i]));
-			}
-    		return rc;
-    	}
+    protected AsciiBuffer createDestination(String name) {
+   		return new AsciiBuffer(new AsciiBuffer(name));
     }
 }

Modified: activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/ConduitBridge.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/ConduitBridge.java?rev=786526&r1=786525&r2=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/ConduitBridge.java (original)
+++ activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/ConduitBridge.java Fri Jun 19 14:54:26 2009
@@ -21,7 +21,7 @@
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.activemq.apollo.broker.wildcard.DestinationFilter;
+import org.apache.activemq.apollo.broker.path.PathFilter;
 import org.apache.activemq.command.ConsumerId;
 import org.apache.activemq.command.ConsumerInfo;
 import org.apache.activemq.transport.Transport;
@@ -59,10 +59,10 @@
     protected boolean addToAlreadyInterestedConsumers(ConsumerInfo info) {
         // search through existing subscriptions and see if we have a match
         boolean matched = false;
-        DestinationFilter filter = DestinationFilter.parseFilter(info.getDestination());
+        PathFilter filter = PathFilter.parseFilter(info.getDestination().getName());
         for (Iterator i = subscriptionMapByLocalId.values().iterator(); i.hasNext();) {
             DemandSubscription ds = (DemandSubscription)i.next();
-            if (filter.matches(ds.getLocalInfo().getDestination())) {
+            if (filter.matches(ds.getLocalInfo().getDestination().getName())) {
                 if (LOG.isDebugEnabled()) {
                     LOG.debug(configuration.getBrokerName() + " matched (add interest) to exsting sub for: " + ds.getRemoteInfo()
                             + " with sub: " + info);

Modified: activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DemandForwardingBridgeSupport.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DemandForwardingBridgeSupport.java?rev=786526&r1=786525&r2=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DemandForwardingBridgeSupport.java (original)
+++ activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DemandForwardingBridgeSupport.java Fri Jun 19 14:54:26 2009
@@ -32,7 +32,7 @@
 
 import org.apache.activemq.Service;
 import org.apache.activemq.advisory.AdvisorySupport;
-import org.apache.activemq.apollo.broker.wildcard.DestinationFilter;
+import org.apache.activemq.apollo.broker.path.PathFilter;
 import org.apache.activemq.command.ActiveMQDestination;
 import org.apache.activemq.command.ActiveMQMessage;
 import org.apache.activemq.command.ActiveMQTempDestination;
@@ -889,21 +889,21 @@
         	}
         } 
 
-        final DestinationFilter filter = DestinationFilter.parseFilter(destination);
+        final PathFilter filter = PathFilter.parseFilter(destination.getName());
         
         ActiveMQDestination[] dests = excludedDestinations;
         if (dests != null && dests.length > 0) {
             for (int i = 0; i < dests.length; i++) {
-                DestinationFilter exclusionFilter = filter;
+                PathFilter exclusionFilter = filter;
                 ActiveMQDestination match = dests[i];
-                if (exclusionFilter instanceof org.apache.activemq.apollo.broker.wildcard.SimpleDestinationFilter) {
-                    DestinationFilter newFilter = DestinationFilter.parseFilter(match);
-                    if (!(newFilter instanceof org.apache.activemq.apollo.broker.wildcard.SimpleDestinationFilter)) {
+                if (exclusionFilter instanceof org.apache.activemq.apollo.broker.path.SimplePathFilter) {
+                    PathFilter newFilter = PathFilter.parseFilter(match.getName());
+                    if (!(newFilter instanceof org.apache.activemq.apollo.broker.path.SimplePathFilter)) {
                         exclusionFilter = newFilter;
                         match = destination;
                     }
                 }
-                if (match != null && exclusionFilter.matches(match) && dests[i].getDestinationType() == destination.getDestinationType()) {
+                if (match != null && exclusionFilter.matches(match.getName()) && dests[i].getDestinationType() == destination.getDestinationType()) {
                     return false;
                 }
             }
@@ -911,16 +911,16 @@
         dests = dynamicallyIncludedDestinations;
         if (dests != null && dests.length > 0) {
             for (int i = 0; i < dests.length; i++) {
-                DestinationFilter inclusionFilter = filter;
+                PathFilter inclusionFilter = filter;
                 ActiveMQDestination match = dests[i];
-                if (inclusionFilter instanceof org.apache.activemq.apollo.broker.wildcard.SimpleDestinationFilter) {
-                    DestinationFilter newFilter = DestinationFilter.parseFilter(match);
-                    if (!(newFilter instanceof org.apache.activemq.apollo.broker.wildcard.SimpleDestinationFilter)) {
+                if (inclusionFilter instanceof org.apache.activemq.apollo.broker.path.SimplePathFilter) {
+                    PathFilter newFilter = PathFilter.parseFilter(match.getName());
+                    if (!(newFilter instanceof org.apache.activemq.apollo.broker.path.SimplePathFilter)) {
                         inclusionFilter = newFilter;
                         match = destination;
                     }
                 }
-                if (match != null && inclusionFilter.matches(match) && dests[i].getDestinationType() == destination.getDestinationType()) {
+                if (match != null && inclusionFilter.matches(match.getName()) && dests[i].getDestinationType() == destination.getDestinationType()) {
                     return true;
                 }
             }

Modified: activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DurableConduitBridge.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DurableConduitBridge.java?rev=786526&r1=786525&r2=786526&view=diff
==============================================================================
--- activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DurableConduitBridge.java (original)
+++ activemq/sandbox/activemq-flow/activemq-network/src/main/java/org/apache/activemq/network/DurableConduitBridge.java Fri Jun 19 14:54:26 2009
@@ -19,7 +19,7 @@
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.activemq.apollo.broker.wildcard.DestinationFilter;
+import org.apache.activemq.apollo.broker.path.PathFilter;
 import org.apache.activemq.command.ActiveMQDestination;
 import org.apache.activemq.command.ConsumerId;
 import org.apache.activemq.command.ConsumerInfo;
@@ -101,10 +101,10 @@
     }
 
     protected boolean doesConsumerExist(ActiveMQDestination dest) {
-        DestinationFilter filter = DestinationFilter.parseFilter(dest);
+        PathFilter filter = PathFilter.parseFilter(dest.getName());
         for (Iterator i = subscriptionMapByLocalId.values().iterator(); i.hasNext();) {
             DemandSubscription ds = (DemandSubscription)i.next();
-            if (filter.matches(ds.getLocalInfo().getDestination())) {
+            if (filter.matches(ds.getLocalInfo().getDestination().getName())) {
                 return true;
             }
         }