You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by js...@apache.org on 2006/01/05 16:36:19 UTC

svn commit: r366204 - in /incubator/activemq/trunk/activemq-core/src: main/java/org/apache/activemq/broker/region/ main/java/org/apache/activemq/broker/region/group/ main/java/org/apache/activemq/selector/ test/java/org/apache/activemq/broker/region/ t...

Author: jstrachan
Date: Thu Jan  5 07:36:07 2006
New Revision: 366204

URL: http://svn.apache.org/viewcvs?rev=366204&view=rev
Log:
minor refactor to allow pluggable MessageGroupMap implementations so that we don't have to keep around every single GroupID in RAM if we don't want to (which could very easily become a RAM leak) and can use a provider which uses hashbuckets instead.
added an initial implementation of a hashbucket based MessageGroupMap which should be fine; am waiting on a unit test run to complete before I dare enable it by default :)

Added:
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java   (with props)
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java   (with props)
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java   (with props)
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java   (with props)
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java   (with props)
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java   (with props)
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html   (with props)
    incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/
    incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/
    incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java   (with props)
    incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java   (with props)
Modified:
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/Queue.java
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/QueueSubscription.java
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParser.java
    incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParserTokenManager.java

Modified: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/Queue.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/Queue.java?rev=366204&r1=366203&r2=366204&view=diff
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/Queue.java (original)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/Queue.java Thu Jan  5 07:36:07 2006
@@ -16,13 +16,12 @@
  */
 package org.apache.activemq.broker.region;
 
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
+import edu.emory.mathcs.backport.java.util.concurrent.CopyOnWriteArrayList;
 
 import org.apache.activemq.broker.ConnectionContext;
+import org.apache.activemq.broker.region.group.MessageGroupMap;
+import org.apache.activemq.broker.region.group.MessageGroupSet;
+import org.apache.activemq.broker.region.group.SimpleMessageGroupMap;
 import org.apache.activemq.broker.region.policy.DeadLetterStrategy;
 import org.apache.activemq.broker.region.policy.DispatchPolicy;
 import org.apache.activemq.broker.region.policy.RoundRobinDispatchPolicy;
@@ -42,8 +41,11 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import edu.emory.mathcs.backport.java.util.concurrent.ConcurrentHashMap;
-import edu.emory.mathcs.backport.java.util.concurrent.CopyOnWriteArrayList;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
 
 /**
  * The Queue is a List of MessageEntry objects that are dispatched to matching
@@ -63,7 +65,7 @@
     protected final DestinationStatistics destinationStatistics = new DestinationStatistics();
 
     private Subscription exclusiveOwner;
-    private final ConcurrentHashMap messageGroupOwners = new ConcurrentHashMap();
+    private final MessageGroupMap messageGroupOwners = new SimpleMessageGroupMap();
 
     protected long garbageSize = 0;
     protected long garbageSizeBeforeCollection = 1000;
@@ -183,16 +185,8 @@
                 wasExclusiveOwner = true;
             }
 
-            HashSet ownedGroups = new HashSet();
             ConsumerId consumerId = sub.getConsumerInfo().getConsumerId();
-            for (Iterator iter = messageGroupOwners.keySet().iterator(); iter.hasNext();) {
-                String group = (String) iter.next();
-                ConsumerId owner = (ConsumerId) messageGroupOwners.get(group);
-                if (owner.equals(consumerId)) {
-                    ownedGroups.add(group);
-                    iter.remove();
-                }
-            }
+            MessageGroupSet ownedGroups = messageGroupOwners.removeConsumer(consumerId);
 
             synchronized (messages) {
                 if (!sub.getConsumerInfo().isBrowser()) {
@@ -305,7 +299,7 @@
 
     public String toString() {
         return "Queue: destination=" + destination.getPhysicalName() + ", subscriptions=" + consumers.size() + ", memory=" + usageManager.getPercentUsage()
-                + "%, size=" + messages.size() + ", in flight groups=" + messageGroupOwners.size();
+                + "%, size=" + messages.size() + ", in flight groups=" + messageGroupOwners;
     }
 
     public void start() throws Exception {
@@ -328,7 +322,7 @@
         return destinationStatistics;
     }
 
-    public ConcurrentHashMap getMessageGroupOwners() {
+    public MessageGroupMap getMessageGroupOwners() {
         return messageGroupOwners;
     }
 

Modified: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/QueueSubscription.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/QueueSubscription.java?rev=366204&r1=366203&r2=366204&view=diff
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/QueueSubscription.java (original)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/QueueSubscription.java Thu Jan  5 07:36:07 2006
@@ -21,6 +21,7 @@
 import javax.jms.InvalidSelectorException;
 
 import org.apache.activemq.broker.ConnectionContext;
+import org.apache.activemq.broker.region.group.MessageGroupMap;
 import org.apache.activemq.command.ConsumerId;
 import org.apache.activemq.command.ConsumerInfo;
 import org.apache.activemq.command.MessageAck;
@@ -77,7 +78,7 @@
         int sequence = node.getGroupSequence();
         if( groupId!=null ) {
             
-            ConcurrentHashMap messageGroupOwners = ((Queue)node.getRegionDestination()).getMessageGroupOwners();            
+            MessageGroupMap messageGroupOwners = ((Queue)node.getRegionDestination()).getMessageGroupOwners();            
             
             // If we can own the first, then no-one else should own the rest.
             if( sequence==1 ) {
@@ -93,7 +94,7 @@
             // need to become the new owner.
             ConsumerId groupOwner;
             synchronized(node) {
-                groupOwner = (ConsumerId) messageGroupOwners.get(groupId);
+                groupOwner = messageGroupOwners.get(groupId);
                 if( groupOwner==null ) {
                     if( node.lock(this) ) {
                         messageGroupOwners.put(groupId, info.getConsumerId());
@@ -107,7 +108,7 @@
             if( groupOwner.equals(info.getConsumerId()) ) {
                 // A group sequence < 1 is an end of group signal.
                 if ( sequence < 1 ) {
-                    messageGroupOwners.remove(groupId);
+                    messageGroupOwners.removeGroup(groupId);
                 }
                 return true;
             }

Added: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java (added)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,32 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+/**
+ * Represents an empty {@link MessageGroupSet}
+ * 
+ * @version $Revision$
+ */
+public class EmptyMessageGroupSet implements MessageGroupSet {
+
+    public static final MessageGroupSet INSTANCE = new EmptyMessageGroupSet();
+    
+    public boolean contains(String groupID) {
+        return false;
+    }
+}

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/EmptyMessageGroupSet.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java (added)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,107 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+import org.apache.activemq.command.ConsumerId;
+
+/**
+ * Uses hash-code buckets to associate consumers with sets of message group IDs.
+ * 
+ * @version $Revision$
+ */
+public class MessageGroupHashBucket implements MessageGroupMap {
+
+    private final int bucketCount;
+    private final ConsumerId[] consumers;
+
+    public MessageGroupHashBucket(int bucketCount) {
+        this.bucketCount = bucketCount;
+        this.consumers = new ConsumerId[bucketCount];
+    }
+
+    public void put(String groupId, ConsumerId consumerId) {
+        int bucket = getBucketNumber(groupId);
+        consumers[bucket] = consumerId;
+    }
+
+    public ConsumerId get(String groupId) {
+        int bucket = getBucketNumber(groupId);
+        return consumers[bucket];
+    }
+
+    public ConsumerId removeGroup(String groupId) {
+        int bucket = getBucketNumber(groupId);
+        ConsumerId answer = consumers[bucket];
+        consumers[bucket] = null;
+        return answer;
+    }
+
+    public MessageGroupSet removeConsumer(ConsumerId consumerId) {
+        MessageGroupSet answer = null;
+        for (int i = 0; i < consumers.length; i++) {
+            ConsumerId owner = consumers[i];
+            if (owner != null && owner.equals(consumerId)) {
+                answer = createMessageGroupSet(i, answer);
+                consumers[i] = null;
+            }
+        }
+        if (answer == null) {
+            // make an empty set
+            answer = EmptyMessageGroupSet.INSTANCE;
+        }
+        return answer;
+    }
+
+    public String toString() {
+        int count = 0;
+        for (int i = 0; i < consumers.length; i++) {
+            if (consumers[i] != null) {
+                count++;
+            }
+        }
+        return "active message group buckets: " + count;
+    }
+
+    protected MessageGroupSet createMessageGroupSet(int bucketNumber, final MessageGroupSet parent) {
+        final MessageGroupSet answer = createMessageGroupSet(bucketNumber);
+        if (parent == null) {
+            return answer;
+        }
+        else {
+            // union the two sets together
+            return new MessageGroupSet() {
+                public boolean contains(String groupID) {
+                    return parent.contains(groupID) || answer.contains(groupID);
+                }
+            };
+        }
+    }
+
+    protected MessageGroupSet createMessageGroupSet(final int bucketNumber) {
+        return new MessageGroupSet() {
+            public boolean contains(String groupID) {
+                int bucket = getBucketNumber(groupID);
+                return bucket == bucketNumber;
+            }
+        };
+    }
+
+    protected int getBucketNumber(String groupId) {
+        return groupId.hashCode() % bucketCount;
+    }
+}

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupHashBucket.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java (added)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,37 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+import org.apache.activemq.command.ConsumerId;
+
+/**
+ * Represents a map of JMSXGroupID values to consumer IDs
+ * 
+ * @version $Revision$
+ */
+public interface MessageGroupMap {
+
+    void put(String groupId, ConsumerId consumerId);
+
+    ConsumerId get(String groupId);
+
+    ConsumerId removeGroup(String groupId);
+
+    MessageGroupSet removeConsumer(ConsumerId consumerId);
+
+}

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupMap.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java (added)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,29 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+/**
+ * Represents a set of Message Group IDs
+ * 
+ * @version $Revision$
+ */
+public interface MessageGroupSet {
+
+    boolean contains(String groupID);
+
+}

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/MessageGroupSet.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java (added)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,66 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+import edu.emory.mathcs.backport.java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.activemq.command.ConsumerId;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * A simple implementation which tracks every individual GroupID value but
+ * which can become a memory leak if clients die before they complete a message
+ * group.
+ * 
+ * @version $Revision$
+ */
+public class SimpleMessageGroupMap implements MessageGroupMap {
+    private Map map = new ConcurrentHashMap();
+    
+    public void put(String groupId, ConsumerId consumerId) {
+        map.put(groupId, consumerId);
+    }
+
+    public ConsumerId get(String groupId) {
+        return (ConsumerId) map.get(groupId);
+    }
+
+    public ConsumerId removeGroup(String groupId) {
+        return (ConsumerId) map.remove(groupId);
+    }
+
+    public MessageGroupSet removeConsumer(ConsumerId consumerId) {
+        SimpleMessageGroupSet ownedGroups = new SimpleMessageGroupSet();
+        for (Iterator iter = map.keySet().iterator(); iter.hasNext();) {
+            String group = (String) iter.next();
+            ConsumerId owner = (ConsumerId) map.get(group);
+            if (owner.equals(consumerId)) {
+                ownedGroups.add(group);
+                iter.remove();
+            }
+        }
+        return ownedGroups;
+    }
+
+    public String toString() {
+        return "message groups: " + map.size();
+    }
+
+}

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupMap.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java (added)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,40 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * A simple implementation which just uses a {@link Set}
+ * 
+ * @version $Revision$
+ */
+public class SimpleMessageGroupSet implements MessageGroupSet {
+
+    private Set set = new HashSet();
+
+    public boolean contains(String groupID) {
+        return set.contains(groupID);
+    }
+
+    public void add(String group) {
+        set.add(group);
+    }
+
+}

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/SimpleMessageGroupSet.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html (added)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html Thu Jan  5 07:36:07 2006
@@ -0,0 +1,9 @@
+<html>
+<head>
+</head>
+<body>
+
+ Classes to implement the <a href="http://activemq.org/Message+Groups">Message Groups</a>a> feature.
+
+</body>
+</html>

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html
------------------------------------------------------------------------------
    svn:executable = *

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/region/group/package.html
------------------------------------------------------------------------------
    svn:mime-type = text/html

Modified: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParser.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParser.java?rev=366204&r1=366203&r2=366204&view=diff
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParser.java (original)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParser.java Thu Jan  5 07:36:07 2006
@@ -518,600 +518,484 @@
 
   final private boolean jj_2_1(int xla) {
     jj_la = xla; jj_lastpos = jj_scanpos = token;
-    return !jj_3_1();
+    try { return !jj_3_1(); }
+    catch(LookaheadSuccess ls) { return true; }
   }
 
   final private boolean jj_2_2(int xla) {
     jj_la = xla; jj_lastpos = jj_scanpos = token;
-    return !jj_3_2();
+    try { return !jj_3_2(); }
+    catch(LookaheadSuccess ls) { return true; }
   }
 
   final private boolean jj_2_3(int xla) {
     jj_la = xla; jj_lastpos = jj_scanpos = token;
-    return !jj_3_3();
+    try { return !jj_3_3(); }
+    catch(LookaheadSuccess ls) { return true; }
   }
 
   final private boolean jj_2_4(int xla) {
     jj_la = xla; jj_lastpos = jj_scanpos = token;
-    return !jj_3_4();
+    try { return !jj_3_4(); }
+    catch(LookaheadSuccess ls) { return true; }
   }
 
   final private boolean jj_2_5(int xla) {
     jj_la = xla; jj_lastpos = jj_scanpos = token;
-    return !jj_3_5();
+    try { return !jj_3_5(); }
+    catch(LookaheadSuccess ls) { return true; }
   }
 
   final private boolean jj_2_6(int xla) {
     jj_la = xla; jj_lastpos = jj_scanpos = token;
-    return !jj_3_6();
+    try { return !jj_3_6(); }
+    catch(LookaheadSuccess ls) { return true; }
   }
 
-  final private boolean jj_3R_58() {
+  final private boolean jj_3R_56() {
     if (jj_scan_token(37)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_11()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_9()) return true;
     return false;
   }
 
-  final private boolean jj_3R_53() {
+  final private boolean jj_3R_51() {
     if (jj_scan_token(32)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     return false;
   }
 
-  final private boolean jj_3R_29() {
+  final private boolean jj_3R_27() {
     if (jj_scan_token(ID)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_63() {
+  final private boolean jj_3R_61() {
     if (jj_scan_token(35)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     return false;
   }
 
-  final private boolean jj_3R_50() {
+  final private boolean jj_3R_48() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_58()) {
+    if (jj_3R_56()) {
     jj_scanpos = xsp;
-    if (jj_3R_59()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_57()) return true;
+    }
     return false;
   }
 
-  final private boolean jj_3R_40() {
+  final private boolean jj_3R_38() {
     if (jj_scan_token(OR)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_39()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_37()) return true;
     return false;
   }
 
-  final private boolean jj_3R_52() {
+  final private boolean jj_3R_50() {
     if (jj_scan_token(31)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     return false;
   }
 
-  final private boolean jj_3R_27() {
+  final private boolean jj_3R_25() {
     if (jj_scan_token(34)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_30()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_28()) return true;
     if (jj_scan_token(36)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_26() {
-    if (jj_3R_29()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_24() {
+    if (jj_3R_27()) return true;
     return false;
   }
 
-  final private boolean jj_3R_51() {
+  final private boolean jj_3R_49() {
     if (jj_scan_token(30)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     return false;
   }
 
-  final private boolean jj_3R_46() {
+  final private boolean jj_3R_44() {
     Token xsp;
     xsp = jj_scanpos;
+    if (jj_3R_49()) {
+    jj_scanpos = xsp;
+    if (jj_3R_50()) {
+    jj_scanpos = xsp;
     if (jj_3R_51()) {
     jj_scanpos = xsp;
     if (jj_3R_52()) {
     jj_scanpos = xsp;
     if (jj_3R_53()) {
     jj_scanpos = xsp;
-    if (jj_3R_54()) {
-    jj_scanpos = xsp;
-    if (jj_3R_55()) {
-    jj_scanpos = xsp;
     if (jj_3_2()) {
     jj_scanpos = xsp;
-    if (jj_3R_56()) {
+    if (jj_3R_54()) {
     jj_scanpos = xsp;
     if (jj_3_3()) {
     jj_scanpos = xsp;
-    if (jj_3R_57()) {
+    if (jj_3R_55()) {
     jj_scanpos = xsp;
     if (jj_3_4()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    }
+    }
+    }
+    }
+    }
+    }
+    }
+    }
+    }
     return false;
   }
 
-  final private boolean jj_3R_25() {
-    if (jj_3R_28()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_23() {
+    if (jj_3R_26()) return true;
     return false;
   }
 
-  final private boolean jj_3R_45() {
-    if (jj_3R_11()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_43() {
+    if (jj_3R_9()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_50()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_48()) { jj_scanpos = xsp; break; }
     }
     return false;
   }
 
-  final private boolean jj_3R_30() {
-    if (jj_3R_39()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_28() {
+    if (jj_3R_37()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_40()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_38()) { jj_scanpos = xsp; break; }
     }
     return false;
   }
 
-  final private boolean jj_3R_24() {
+  final private boolean jj_3R_22() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_25()) {
+    if (jj_3R_23()) {
     jj_scanpos = xsp;
-    if (jj_3R_26()) {
+    if (jj_3R_24()) {
     jj_scanpos = xsp;
-    if (jj_3R_27()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_25()) return true;
+    }
+    }
     return false;
   }
 
-  final private boolean jj_3R_19() {
-    if (jj_3R_24()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_17() {
+    if (jj_3R_22()) return true;
     return false;
   }
 
-  final private boolean jj_3R_23() {
+  final private boolean jj_3R_21() {
     if (jj_scan_token(STRING_LITERAL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_62() {
+  final private boolean jj_3R_60() {
     if (jj_scan_token(35)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     return false;
   }
 
-  final private boolean jj_3R_18() {
+  final private boolean jj_3R_16() {
     if (jj_scan_token(XQUERY)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     return false;
   }
 
-  final private boolean jj_3R_43() {
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_41() {
+    if (jj_3R_43()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_46()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_44()) { jj_scanpos = xsp; break; }
     }
     return false;
   }
 
-  final private boolean jj_3R_61() {
+  final private boolean jj_3R_59() {
     if (jj_scan_token(ESCAPE)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     return false;
   }
 
   final private boolean jj_3_4() {
     if (jj_scan_token(NOT)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(IN)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(34)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_63()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_61()) { jj_scanpos = xsp; break; }
     }
     if (jj_scan_token(36)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
   final private boolean jj_3_6() {
     if (jj_scan_token(37)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_10()) return true;
     return false;
   }
 
-  final private boolean jj_3R_17() {
+  final private boolean jj_3R_15() {
     if (jj_scan_token(XPATH)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     return false;
   }
 
-  final private boolean jj_3R_38() {
-    if (jj_scan_token(NULL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_14() {
+    if (jj_scan_token(NOT)) return true;
+    if (jj_3R_10()) return true;
     return false;
   }
 
-  final private boolean jj_3R_16() {
-    if (jj_scan_token(NOT)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_36() {
+    if (jj_scan_token(NULL)) return true;
     return false;
   }
 
-  final private boolean jj_3R_14() {
+  final private boolean jj_3R_12() {
     if (jj_scan_token(37)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_10()) return true;
     return false;
   }
 
-  final private boolean jj_3R_57() {
+  final private boolean jj_3R_55() {
     if (jj_scan_token(IN)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(34)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_62()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_60()) { jj_scanpos = xsp; break; }
     }
     if (jj_scan_token(36)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_49() {
+  final private boolean jj_3R_47() {
     if (jj_scan_token(IS)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(NOT)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(NULL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_15() {
+  final private boolean jj_3R_13() {
     if (jj_scan_token(38)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_10()) return true;
     return false;
   }
 
-  final private boolean jj_3R_37() {
+  final private boolean jj_3R_35() {
     if (jj_scan_token(FALSE)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
   final private boolean jj_3_1() {
     if (jj_scan_token(IS)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(NULL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_12() {
+  final private boolean jj_3R_10() {
     Token xsp;
     xsp = jj_scanpos;
+    if (jj_3R_12()) {
+    jj_scanpos = xsp;
+    if (jj_3R_13()) {
+    jj_scanpos = xsp;
     if (jj_3R_14()) {
     jj_scanpos = xsp;
     if (jj_3R_15()) {
     jj_scanpos = xsp;
     if (jj_3R_16()) {
     jj_scanpos = xsp;
-    if (jj_3R_17()) {
-    jj_scanpos = xsp;
-    if (jj_3R_18()) {
-    jj_scanpos = xsp;
-    if (jj_3R_19()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_17()) return true;
+    }
+    }
+    }
+    }
+    }
     return false;
   }
 
-  final private boolean jj_3R_36() {
+  final private boolean jj_3R_34() {
     if (jj_scan_token(TRUE)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_48() {
+  final private boolean jj_3R_46() {
     if (jj_scan_token(29)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_43()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_41()) return true;
     return false;
   }
 
   final private boolean jj_3_3() {
     if (jj_scan_token(NOT)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(BETWEEN)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     if (jj_scan_token(AND)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     return false;
   }
 
-  final private boolean jj_3R_47() {
+  final private boolean jj_3R_45() {
     if (jj_scan_token(28)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_43()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_41()) return true;
     return false;
   }
 
-  final private boolean jj_3R_44() {
+  final private boolean jj_3R_42() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_47()) {
+    if (jj_3R_45()) {
     jj_scanpos = xsp;
-    if (jj_3R_48()) {
+    if (jj_3R_46()) {
     jj_scanpos = xsp;
     if (jj_3_1()) {
     jj_scanpos = xsp;
-    if (jj_3R_49()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_47()) return true;
+    }
+    }
+    }
     return false;
   }
 
-  final private boolean jj_3R_35() {
+  final private boolean jj_3R_33() {
     if (jj_scan_token(FLOATING_POINT_LITERAL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_56() {
+  final private boolean jj_3R_54() {
     if (jj_scan_token(BETWEEN)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     if (jj_scan_token(AND)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     return false;
   }
 
-  final private boolean jj_3R_60() {
+  final private boolean jj_3R_58() {
     if (jj_scan_token(ESCAPE)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     return false;
   }
 
-  final private boolean jj_3R_34() {
+  final private boolean jj_3R_32() {
     if (jj_scan_token(OCTAL_LITERAL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_22() {
+  final private boolean jj_3R_20() {
     if (jj_scan_token(41)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_10()) return true;
     return false;
   }
 
-  final private boolean jj_3R_41() {
-    if (jj_3R_43()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_39() {
+    if (jj_3R_41()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_44()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_42()) { jj_scanpos = xsp; break; }
     }
     return false;
   }
 
   final private boolean jj_3_2() {
     if (jj_scan_token(NOT)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     if (jj_scan_token(LIKE)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_61()) jj_scanpos = xsp;
-    else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_59()) jj_scanpos = xsp;
     return false;
   }
 
-  final private boolean jj_3R_55() {
+  final private boolean jj_3R_53() {
     if (jj_scan_token(LIKE)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_21()) return true;
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_60()) jj_scanpos = xsp;
-    else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_58()) jj_scanpos = xsp;
     return false;
   }
 
-  final private boolean jj_3R_33() {
+  final private boolean jj_3R_31() {
     if (jj_scan_token(HEX_LITERAL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_21() {
+  final private boolean jj_3R_19() {
     if (jj_scan_token(40)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_10()) return true;
     return false;
   }
 
-  final private boolean jj_3R_13() {
+  final private boolean jj_3R_18() {
+    if (jj_scan_token(39)) return true;
+    if (jj_3R_10()) return true;
+    return false;
+  }
+
+  final private boolean jj_3R_11() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_20()) {
+    if (jj_3R_18()) {
     jj_scanpos = xsp;
-    if (jj_3R_21()) {
+    if (jj_3R_19()) {
     jj_scanpos = xsp;
-    if (jj_3R_22()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    return false;
-  }
-
-  final private boolean jj_3R_20() {
-    if (jj_scan_token(39)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_20()) return true;
+    }
+    }
     return false;
   }
 
-  final private boolean jj_3R_42() {
+  final private boolean jj_3R_40() {
     if (jj_scan_token(AND)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_41()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_39()) return true;
     return false;
   }
 
-  final private boolean jj_3R_32() {
+  final private boolean jj_3R_30() {
     if (jj_scan_token(DECIMAL_LITERAL)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
     return false;
   }
 
-  final private boolean jj_3R_11() {
-    if (jj_3R_12()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_9() {
+    if (jj_3R_10()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_13()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_11()) { jj_scanpos = xsp; break; }
     }
     return false;
   }
 
-  final private boolean jj_3R_10() {
-    if (jj_scan_token(38)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    return false;
-  }
-
-  final private boolean jj_3R_31() {
-    if (jj_3R_23()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    return false;
-  }
-
-  final private boolean jj_3R_9() {
-    if (jj_scan_token(37)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_29() {
+    if (jj_3R_21()) return true;
     return false;
   }
 
-  final private boolean jj_3R_59() {
+  final private boolean jj_3R_57() {
     if (jj_scan_token(38)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_11()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_9()) return true;
     return false;
   }
 
-  final private boolean jj_3R_39() {
-    if (jj_3R_41()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+  final private boolean jj_3R_37() {
+    if (jj_3R_39()) return true;
     Token xsp;
     while (true) {
       xsp = jj_scanpos;
-      if (jj_3R_42()) { jj_scanpos = xsp; break; }
-      if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+      if (jj_3R_40()) { jj_scanpos = xsp; break; }
     }
     return false;
   }
@@ -1119,19 +1003,21 @@
   final private boolean jj_3_5() {
     Token xsp;
     xsp = jj_scanpos;
-    if (jj_3R_9()) {
+    if (jj_scan_token(37)) {
     jj_scanpos = xsp;
-    if (jj_3R_10()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_11()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_scan_token(38)) return true;
+    }
+    if (jj_3R_9()) return true;
     return false;
   }
 
-  final private boolean jj_3R_28() {
+  final private boolean jj_3R_26() {
     Token xsp;
     xsp = jj_scanpos;
+    if (jj_3R_29()) {
+    jj_scanpos = xsp;
+    if (jj_3R_30()) {
+    jj_scanpos = xsp;
     if (jj_3R_31()) {
     jj_scanpos = xsp;
     if (jj_3R_32()) {
@@ -1142,27 +1028,20 @@
     jj_scanpos = xsp;
     if (jj_3R_35()) {
     jj_scanpos = xsp;
-    if (jj_3R_36()) {
-    jj_scanpos = xsp;
-    if (jj_3R_37()) {
-    jj_scanpos = xsp;
-    if (jj_3R_38()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    } else if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_36()) return true;
+    }
+    }
+    }
+    }
+    }
+    }
+    }
     return false;
   }
 
-  final private boolean jj_3R_54() {
+  final private boolean jj_3R_52() {
     if (jj_scan_token(33)) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
-    if (jj_3R_45()) return true;
-    if (jj_la == 0 && jj_scanpos == jj_lastpos) return false;
+    if (jj_3R_43()) return true;
     return false;
   }
 
@@ -1227,6 +1106,8 @@
     throw generateParseException();
   }
 
+  static private final class LookaheadSuccess extends java.lang.Error { }
+  final private LookaheadSuccess jj_ls = new LookaheadSuccess();
   final private boolean jj_scan_token(int kind) {
     if (jj_scanpos == jj_lastpos) {
       jj_la--;
@@ -1238,7 +1119,9 @@
     } else {
       jj_scanpos = jj_scanpos.next;
     }
-    return (jj_scanpos.kind != kind);
+    if (jj_scanpos.kind != kind) return true;
+    if (jj_la == 0 && jj_scanpos == jj_lastpos) throw jj_ls;
+    return false;
   }
 
   final public Token getNextToken() {
@@ -1264,7 +1147,7 @@
       return (jj_ntk = jj_nt.kind);
   }
 
-  final public ParseException generateParseException() {
+  public ParseException generateParseException() {
     Token errortok = token.next;
     int line = errortok.beginLine, column = errortok.beginColumn;
     String mess = (errortok.kind == 0) ? tokenImage[0] : errortok.image;

Modified: incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParserTokenManager.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParserTokenManager.java?rev=366204&r1=366203&r2=366204&view=diff
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParserTokenManager.java (original)
+++ incubator/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/selector/SelectorParserTokenManager.java Thu Jan  5 07:36:07 2006
@@ -904,7 +904,7 @@
 static final long[] jjtoSpecial = {
    0x3eL, 
 };
-private SimpleCharStream input_stream;
+protected SimpleCharStream input_stream;
 private final int[] jjrounds = new int[43];
 private final int[] jjstateSet = new int[86];
 protected char curChar;
@@ -946,7 +946,7 @@
       curLexState = lexState;
 }
 
-private final Token jjFillToken()
+protected Token jjFillToken()
 {
    Token t = Token.newToken(jjmatchedKind);
    t.kind = jjmatchedKind;
@@ -966,7 +966,7 @@
 int jjmatchedPos;
 int jjmatchedKind;
 
-public final Token getNextToken() 
+public Token getNextToken() 
 {
   int kind;
   Token specialToken = null;

Added: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java (added)
+++ incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,31 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+
+/**
+ *
+ * @version $Revision$
+ */
+public class MessageGroupHashBucketTest extends MessageGroupMapTest {
+
+    protected MessageGroupMap createMessageGroupMap() {
+        return new MessageGroupHashBucket(1024);
+    }
+
+}

Propchange: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupHashBucketTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java
URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java?rev=366204&view=auto
==============================================================================
--- incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java (added)
+++ incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java Thu Jan  5 07:36:07 2006
@@ -0,0 +1,106 @@
+/**
+ * 
+ * Copyright 2005 LogicBlaze, Inc. http://www.logicblaze.com
+ * 
+ * Licensed 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.broker.region.group;
+
+import org.apache.activemq.command.ConnectionId;
+import org.apache.activemq.command.ConsumerId;
+import org.apache.activemq.command.SessionId;
+
+import junit.framework.TestCase;
+
+/**
+ * 
+ * @version $Revision$
+ */
+public class MessageGroupMapTest extends TestCase {
+
+    protected MessageGroupMap map;
+    private ConsumerId consumer1;
+    private ConsumerId consumer2;
+    private ConsumerId consumer3;
+    private long idCounter;
+
+    public void testSingleConsumerForManyBucks() throws Exception {
+        assertGet("1", null);
+
+        map.put("1", consumer1);
+        assertGet("1", consumer1);
+        map.put("2", consumer1);
+        assertGet("2", consumer1);
+        map.put("3", consumer1);
+        assertGet("3", consumer1);
+
+        MessageGroupSet set = map.removeConsumer(consumer1);
+        assertContains(set, "1");
+        assertContains(set, "2");
+        assertContains(set, "3");
+        assertGet("1", null);
+        assertGet("2", null);
+        assertGet("3", null);
+    }
+
+    public void testManyConsumers() throws Exception {
+        assertGet("1", null);
+
+        map.put("1", consumer1);
+        assertGet("1", consumer1);
+        map.put("2", consumer2);
+        assertGet("2", consumer2);
+        map.put("3", consumer3);
+        assertGet("3", consumer3);
+
+        MessageGroupSet set = map.removeConsumer(consumer1);
+        assertContains(set, "1");
+
+        assertGet("1", null);
+        map.put("1", consumer2);
+        assertGet("1", consumer2);
+
+        set = map.removeConsumer(consumer2);
+        assertContains(set, "1");
+        assertContains(set, "2");
+    }
+
+    protected void setUp() throws Exception {
+        super.setUp();
+        map = createMessageGroupMap();
+        consumer1 = createConsumerId();
+        consumer2 = createConsumerId();
+        consumer3 = createConsumerId();
+    }
+
+    protected MessageGroupMap createMessageGroupMap() {
+        return new SimpleMessageGroupMap();
+    }
+
+    protected ConsumerId createConsumerId() {
+        ConnectionId connectionId = new ConnectionId("" + ++idCounter);
+        SessionId sessionId = new SessionId(connectionId, ++idCounter);
+        ConsumerId answer = new ConsumerId(sessionId, ++idCounter);
+        return answer;
+    }
+
+    protected void assertGet(String groupdId, ConsumerId expected) {
+        ConsumerId actual = map.get(groupdId);
+        assertEquals("Entry for groupId: " + groupdId, expected, actual);
+    }
+
+    protected void assertContains(MessageGroupSet set, String groupID) {
+        assertTrue("MessageGroup set: " + set + " does not contain groupID: " + groupID, set.contains(groupID));
+    }
+}

Propchange: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Propchange: incubator/activemq/trunk/activemq-core/src/test/java/org/apache/activemq/broker/region/group/MessageGroupMapTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain