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 2011/06/29 19:19:55 UTC

svn commit: r1141176 - in /activemq/activemq-apollo/trunk: apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/ apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/dto/ apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/ ...

Author: chirino
Date: Wed Jun 29 17:19:55 2011
New Revision: 1141176

URL: http://svn.apache.org/viewvc?rev=1141176&view=rev
Log:
Fixes https://issues.apache.org/jira/browse/APLO-54 and https://issues.apache.org/jira/browse/APLO-60: Support regular expressions in destination wildcards and wildcards like "some*foo"

Added:
    activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/RegexChildPathNode.scala
      - copied, changed from r1141174, activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala
Modified:
    activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala
    activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/dto/StompDTO.java
    activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/AnyChildPathNode.scala
    activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala
    activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathMapNode.scala
    activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathParser.scala
    activemq/activemq-apollo/trunk/apollo-util/src/test/scala/org/apache/activemq/apollo/util/path/PathMapTest.scala
    activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md

Modified: activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocolHandler.scala Wed Jun 29 17:19:55 2011
@@ -404,7 +404,10 @@ class StompProtocolHandler extends Proto
         config.destination_separator!=null ||
         config.path_separator!= null ||
         config.any_child_wildcard != null ||
-        config.any_descendant_wildcard!= null ) {
+        config.any_descendant_wildcard!= null ||
+        config.regex_wildcard_start!= null ||
+        config.regex_wildcard_end!= null
+    ) {
 
       destination_parser = new DestinationParser().copy(Stomp.destination_parser)
       if( config.queue_prefix!=null ) { destination_parser.queue_prefix = config.queue_prefix }
@@ -413,6 +416,8 @@ class StompProtocolHandler extends Proto
       if( config.path_separator!=null ) { destination_parser.path_separator = config.path_separator }
       if( config.any_child_wildcard!=null ) { destination_parser.any_child_wildcard = config.any_child_wildcard }
       if( config.any_descendant_wildcard!=null ) { destination_parser.any_descendant_wildcard = config.any_descendant_wildcard }
+      if( config.regex_wildcard_start!=null ) { destination_parser.regex_wildcard_start = config.regex_wildcard_start }
+      if( config.regex_wildcard_end!=null ) { destination_parser.regex_wildcard_end = config.regex_wildcard_end }
 
     }
 

Modified: activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/dto/StompDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/dto/StompDTO.java?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/dto/StompDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/dto/StompDTO.java Wed Jun 29 17:19:55 2011
@@ -74,11 +74,17 @@ public class StompDTO extends ProtocolDT
     @XmlAttribute(name="any_descendant_wildcard")
     public String any_descendant_wildcard;
 
+    @XmlAttribute(name="regex_wildcard_start")
+    public String regex_wildcard_start;
+
+    @XmlAttribute(name="regex_wildcard_end")
+    public String regex_wildcard_end;
 
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;
-        if (!(o instanceof StompDTO)) return false;
+        if (o == null || getClass() != o.getClass()) return false;
+        if (!super.equals(o)) return false;
 
         StompDTO stompDTO = (StompDTO) o;
 
@@ -104,6 +110,10 @@ public class StompDTO extends ProtocolDT
             return false;
         if (queue_prefix != null ? !queue_prefix.equals(stompDTO.queue_prefix) : stompDTO.queue_prefix != null)
             return false;
+        if (regex_wildcard_end != null ? !regex_wildcard_end.equals(stompDTO.regex_wildcard_end) : stompDTO.regex_wildcard_end != null)
+            return false;
+        if (regex_wildcard_start != null ? !regex_wildcard_start.equals(stompDTO.regex_wildcard_start) : stompDTO.regex_wildcard_start != null)
+            return false;
         if (topic_prefix != null ? !topic_prefix.equals(stompDTO.topic_prefix) : stompDTO.topic_prefix != null)
             return false;
 
@@ -112,7 +122,8 @@ public class StompDTO extends ProtocolDT
 
     @Override
     public int hashCode() {
-        int result = add_user_header != null ? add_user_header.hashCode() : 0;
+        int result = super.hashCode();
+        result = 31 * result + (add_user_header != null ? add_user_header.hashCode() : 0);
         result = 31 * result + (add_user_headers != null ? add_user_headers.hashCode() : 0);
         result = 31 * result + (max_header_length != null ? max_header_length.hashCode() : 0);
         result = 31 * result + (max_headers != null ? max_headers.hashCode() : 0);
@@ -124,6 +135,8 @@ public class StompDTO extends ProtocolDT
         result = 31 * result + (path_separator != null ? path_separator.hashCode() : 0);
         result = 31 * result + (any_child_wildcard != null ? any_child_wildcard.hashCode() : 0);
         result = 31 * result + (any_descendant_wildcard != null ? any_descendant_wildcard.hashCode() : 0);
+        result = 31 * result + (regex_wildcard_start != null ? regex_wildcard_start.hashCode() : 0);
+        result = 31 * result + (regex_wildcard_end != null ? regex_wildcard_end.hashCode() : 0);
         return result;
     }
 }

Modified: activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/AnyChildPathNode.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/AnyChildPathNode.scala?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/AnyChildPathNode.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/AnyChildPathNode.scala Wed Jun 29 17:19:55 2011
@@ -26,11 +26,7 @@ import collection.JavaConversions._
   * ignoring the name of the current path (so for navigating using * in a wildcard).
   *
   */
-class AnyChildPathNode[Value] extends PathNode[Value] {
-  def this(node: PathNode[Value]) {
-    this ()
-    this.node = node
-  }
+class AnyChildPathNode[Value](val node: PathNode[Value]) extends PathNode[Value] {
 
   def appendMatchingValues(answer: Set[Value], path: Path, startIndex: Int): Unit = {
     for (child <- getChildNodes) {
@@ -112,5 +108,4 @@ class AnyChildPathNode[Value] extends Pa
     return node.getChildren
   }
 
-  private var node: PathNode[Value] = null
 }
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala Wed Jun 29 17:19:55 2011
@@ -16,6 +16,8 @@
   */
 package org.apache.activemq.apollo.util.path
 
+import java.util.regex.Pattern
+
 /**
   * Holds the delimiters used to parse paths.
   *
@@ -35,6 +37,8 @@ object RootPart extends Part {
 object AnyChildPart extends Part
 object AnyDescendantPart extends Part
 
+case class RegexChildPart(regex:Pattern, original:String) extends Part
+
 case class LiteralPart(value: String) extends Part {
   override def matches(p: Part) = p match {
     case LiteralPart(v) => v == value

Modified: activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathMapNode.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathMapNode.scala?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathMapNode.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathMapNode.scala Wed Jun 29 17:19:55 2011
@@ -30,16 +30,16 @@ import collection.JavaConversions._
   *
   * @version $Revision: 1.2 $
   */
-class PathMapNode[Value] extends PathNode[Value] {
-  def this(parent: PathMapNode[Value]) {
-    this ()
-    this.parent = parent
-    if (parent == null) {
-      pathLength = 0
-    }
-    else {
-      pathLength = parent.pathLength + 1
-    }
+class PathMapNode[Value](val parent: PathMapNode[Value]) extends PathNode[Value] {
+
+  val childNodes: Map[Part, PathNode[Value]] = new HashMap[Part, PathNode[Value]]
+  val values: List[Value] = new ArrayList[Value]
+  var part: Part = RootPart
+
+  var pathLength:Int = if (parent == null) {
+    0
+  } else {
+    parent.pathLength + 1
   }
 
   /**
@@ -144,6 +144,10 @@ class PathMapNode[Value] extends PathNod
           node.appendMatchingWildcards(answer, path, i)
           node = new AnyChildPathNode[Value](node)
           i += 1;
+        case RegexChildPart(r, _) =>
+          node.appendMatchingWildcards(answer, path, i)
+          node = new RegexChildPathNode[Value](node, r)
+          i += 1;
         case part =>
           node.appendMatchingWildcards(answer, path, i)
           node = node.getChild(part)
@@ -176,13 +180,15 @@ class PathMapNode[Value] extends PathNod
     if (idx - 1 > pathLength) {
       return
     }
-    var wildCardNode: PathMapNode[Value] = getChild(AnyChildPart)
-    if (wildCardNode != null) {
-      wildCardNode.appendMatchingValues(answer, parts, idx + 1)
-    }
-    wildCardNode = getChild(AnyDescendantPart)
-    if (wildCardNode != null) {
-      answer.addAll(wildCardNode.getDesendentValues)
+
+    childNodes.foreach { case (path,node) =>
+      path match {
+        case AnyChildPart => node.appendMatchingValues(answer, parts, idx + 1)
+        case x:RegexChildPart => node.appendMatchingValues(answer, parts, idx + 1)
+        case AnyDescendantPart => answer.addAll(node.getDesendentValues)
+        case x:LiteralPart =>
+        case RootPart =>
+      }
     }
   }
 
@@ -193,19 +199,24 @@ class PathMapNode[Value] extends PathNod
     var i: Int = startIndex
     while (i < size && node != null) {
       var part: Part = path.parts.get(i)
-      if (part == AnyDescendantPart) {
-        answer.addAll(node.getDesendentValues)
-        couldMatchAny = false
-        i = size
-      } else {
-        node.appendMatchingWildcards(answer, path, i)
-        if (part == AnyChildPart) {
+      part match {
+        case AnyDescendantPart =>
+          answer.addAll(node.getDesendentValues)
+          i += size
+          couldMatchAny = false
+        case AnyChildPart =>
+          node.appendMatchingWildcards(answer, path, i)
+          i += 1
           node = new AnyChildPathNode[Value](node)
-        }
-        else {
+        case RegexChildPart(r, _) =>
+          node.appendMatchingWildcards(answer, path, i)
+          i += 1
+          node = new RegexChildPathNode[Value](node, r)
+        case x:LiteralPart =>
+          node.appendMatchingWildcards(answer, path, i)
+          i += 1;
           node = node.getChild(part)
-        }
-        i += 1;
+        case RootPart =>
       }
     }
     if (node != null) {
@@ -234,9 +245,4 @@ class PathMapNode[Value] extends PathNod
     pruneIfEmpty
   }
 
-  private var parent: PathMapNode[Value] = null
-  private var values: List[Value] = new ArrayList[Value]
-  private var childNodes: Map[Part, PathNode[Value]] = new HashMap[Part, PathNode[Value]]
-  private var part: Part = RootPart
-  private var pathLength: Int = 0
 }
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathParser.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathParser.scala?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathParser.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/PathParser.scala Wed Jun 29 17:19:55 2011
@@ -20,6 +20,9 @@ import java.util.LinkedList
 import java.util.regex._
 import collection.JavaConversions._
 import org.apache.activemq.apollo.util.path.PathParser.PartFilter
+import util.matching.Regex
+import collection.mutable.ListBuffer
+
 /**
   * Holds the delimiters used to parse paths.
   *
@@ -33,6 +36,8 @@ object PathParser {
           return true
         case AnyChildPart =>
           return true
+        case x:RegexChildPart =>
+          return true
         case _ =>
       }
     }
@@ -66,6 +71,31 @@ object PathParser {
 
   }
 
+  class RegexChildPathFilter(val regex:Pattern,  next: PartFilter) extends PartFilter {
+
+    def matches(remaining: LinkedList[Part]): Boolean = {
+      if (!remaining.isEmpty) {
+        var p: Part = remaining.removeFirst
+        p match {
+          case LiteralPart(v)=>
+            if ( regex.matcher(v).matches ) {
+              if (next != null) {
+                return next.matches(remaining)
+              } else {
+                return remaining.isEmpty
+              }
+            } else {
+              false
+            }
+          case _ => false
+        }
+      } else {
+        return false
+      }
+    }
+
+  }
+
   class AnyDecendentPathFilter(val next: PartFilter) extends PartFilter {
     def matches(remaining: LinkedList[Part]): Boolean = {
       if (!remaining.isEmpty) {
@@ -84,6 +114,8 @@ class PathParser {
 
   var any_descendant_wildcard = "**"
   var any_child_wildcard = "*"
+  var regex_wildcard_start = "{"
+  var regex_wildcard_end = "}"
   var path_separator = "."
   var part_pattern = Pattern.compile("[a-zA-Z0-9\\_\\-\\%\\~]+")
 
@@ -109,6 +141,19 @@ class PathParser {
     return decode_path(parts(subject))
   }
 
+  def regex_map[T](text:String, pattern: Pattern)(func: Either[CharSequence, Matcher] => T) = {
+    var lastIndex = 0;
+    val m = pattern.matcher(text);
+    val rc = new ListBuffer[T]();
+    while (m.find()) {
+      rc += func(Left(text.subSequence(lastIndex, m.start)))
+      rc += func(Right(m))
+      lastIndex = m.end
+    }
+    rc += func(Left(text.subSequence(lastIndex,  text.length)))
+    rc.toList
+  }
+
   private def decode_part(value: String): Part = {
     if (value == any_child_wildcard) {
       return AnyChildPart
@@ -118,7 +163,33 @@ class PathParser {
       if (part_pattern == null || part_pattern.matcher(value.toString).matches) {
         return LiteralPart(value)
       } else {
-        throw new PathParser.PathException(String.format("Invalid destination path part: '%s', it does not match regex: %s", value, part_pattern))
+
+        val pattern = (
+            (Pattern.quote(regex_wildcard_start)+"(.*?)"+Pattern.quote(regex_wildcard_end)) +
+            "|" +
+            Pattern.quote(any_child_wildcard)
+          ).r.pattern
+
+        val regex = regex_map(value, pattern) { _ match {
+          case Left(x) =>
+            if (x=="") {
+              ""
+            } else {
+              if( part_pattern.matcher(x).matches ) {
+                Pattern.quote(x.toString)
+              } else {
+                throw new PathParser.PathException(String.format("Invalid destination: '%s', it does not match regex: %s", value, part_pattern))
+              }
+            }
+          case Right(wildcard) =>
+            if ( wildcard.group() == any_child_wildcard ) {
+              ".*?"
+            } else {
+              wildcard.group(1)
+            }
+        } }.mkString("")
+
+        return RegexChildPart(("^"+regex+"$").r.pattern, value)
       }
     }
   }
@@ -134,6 +205,7 @@ class PathParser {
       case RootPart => ""
       case AnyChildPart => any_child_wildcard
       case AnyDescendantPart => any_descendant_wildcard
+      case RegexChildPart(_, original) => original
       case LiteralPart(value) => value
     })).toArray
   }
@@ -157,6 +229,8 @@ class PathParser {
           last = new LitteralPathFilter(last, p)
         case AnyChildPart =>
           last = new PathParser.AnyChildPathFilter(last)
+        case RegexChildPart(r, _) =>
+          last = new PathParser.RegexChildPathFilter(r, last)
         case AnyDescendantPart =>
           last = new PathParser.AnyDecendentPathFilter(last)
         case _ =>

Copied: activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/RegexChildPathNode.scala (from r1141174, activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/RegexChildPathNode.scala?p2=activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/RegexChildPathNode.scala&p1=activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala&r1=1141174&r2=1141176&rev=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/Part.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/path/RegexChildPathNode.scala Wed Jun 29 17:19:55 2011
@@ -1,3 +1,5 @@
+package org.apache.activemq.apollo.util.path
+
 /**
   * Licensed to the Apache Software Foundation (ASF) under one or more
   * contributor license agreements.  See the NOTICE file distributed with
@@ -14,30 +16,30 @@
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
-package org.apache.activemq.apollo.util.path
+import java.util.Collection
+import java.util.regex.Pattern
+import collection.JavaConversions._
 
 /**
-  * Holds the delimiters used to parse paths.
+  * 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).
   *
-  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
   */
-sealed trait Part {
-  def matches(p: Part) = true
-}
+class RegexChildPathNode[Value](node: PathNode[Value], val regex:Pattern) extends AnyChildPathNode[Value](node) {
 
-object RootPart extends Part {
-  override def matches(p: Part) = p match {
-    case RootPart => true
-    case _ => false
+  override protected def getChildNodes: Collection[PathNode[Value]] = {
+    super.getChildNodes.filter { child =>
+      child match {
+        case child:PathMapNode[Value] =>
+          child.getPart match {
+            case LiteralPart(name)=>
+              regex.matcher(name).matches()
+            case _ =>
+              false
+          }
+        case _ => false
+      }
+    }
   }
-}
 
-object AnyChildPart extends Part
-object AnyDescendantPart extends Part
-
-case class LiteralPart(value: String) extends Part {
-  override def matches(p: Part) = p match {
-    case LiteralPart(v) => v == value
-    case _ => true
-  }
-}
+}
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-util/src/test/scala/org/apache/activemq/apollo/util/path/PathMapTest.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-util/src/test/scala/org/apache/activemq/apollo/util/path/PathMapTest.scala?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-util/src/test/scala/org/apache/activemq/apollo/util/path/PathMapTest.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-util/src/test/scala/org/apache/activemq/apollo/util/path/PathMapTest.scala Wed Jun 29 17:19:55 2011
@@ -134,6 +134,30 @@ class PathMapTest {
     assertMapValue(map, "TEST.BAR.*", v2, v5, v6)
   }
 
+  @Test def testCustomRegexWildcardPaths: Unit = {
+    var map: PathMap[String] = new PathMap[String]
+    put(map, "TEST.a{[0-9]+}a", v1)
+    put(map, "TEST.BAR.aa", v2)
+    put(map, "TEST.BAR.a123a", v3)
+    put(map, "TEST.BAR.a2ca", v4)
+    put(map, "TEST.BAR.aba", v5)
+
+    assertMapValue(map, "TEST.a99a", v1)
+    assertMapValue(map, "TEST.BAR.a{[0-9]+}a", v3)
+  }
+
+  @Test def testRegexWildcardPaths: Unit = {
+    var map: PathMap[String] = new PathMap[String]
+    put(map, "TEST.a*a", v1)
+    put(map, "TEST.BAR.aa", v2)
+    put(map, "TEST.BAR.aba", v3)
+    put(map, "TEST.BAR.cat", v4)
+    put(map, "TEST.BAR.a", v5)
+
+    assertMapValue(map, "TEST.aba", v1)
+    assertMapValue(map, "TEST.BAR.a*a", v2, v3)
+  }
+
   @Test def testDoubleWildcardDoesNotMatchLongerPattern: Unit = {
     var map: PathMap[String] = new PathMap[String]
     put(map, "TEST.*", v1)

Modified: activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md?rev=1141176&r1=1141175&r2=1141176&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md (original)
+++ activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md Wed Jun 29 17:19:55 2011
@@ -963,6 +963,8 @@ headers are parsed and interpreted.  The
 * `path_separator` : Defaults to `.`
 * `destination_separator` : Defaults to `,`
 * `any_child_wildcard` : Defaults to `*`
+* `regex_wildcard_start` : Defaults to `{`
+* `regex_wildcard_end` : Defaults to `}`
 * `any_descendant_wildcard` : Defaults to `**`
 
 It also supports nested `add_user_header` elements to more finely control how
@@ -1248,6 +1250,7 @@ information you're interested in.
 
 * `.` is used to separate names in a path
 * `*` is used to match any name in a path
+* `{regex}` is used to match a path name against a regular expression.
 * `**` is used to recursively match path names
 
 For example imagine you are sending price messages from a stock exchange feed.
@@ -1266,6 +1269,8 @@ For example using the example above, the
 * `/topic/PRICE.STOCK.**` : Any price for a stock on any exchange
 * `/topic/PRICE.STOCK.NASDAQ.*` : Any stock price on NASDAQ
 * `/topic/PRICE.STOCK.*.IBM` : Any IBM stock price on any exchange
+* `/topic/PRICE.STOCK.*.I*` : Any stock price starting with 'I' on any exchange
+* `/topic/PRICE.STOCK.*.*{[0-9]}` : Any stock price that ends in a digit on any exchange
 
 Destination wildcards can only be used in a SUBSCRIBE frame.