You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2011/09/10 07:43:16 UTC

svn commit: r1167462 [1/3] - in /camel/trunk/camel-core/src: main/java/org/apache/camel/builder/ main/java/org/apache/camel/language/simple/ main/java/org/apache/camel/language/simple/ast/ test/java/org/apache/camel/component/bean/ test/java/org/apache...

Author: davsclaus
Date: Sat Sep 10 05:43:14 2011
New Revision: 1167462

URL: http://svn.apache.org/viewvc?rev=1167462&view=rev
Log:
CAMEL-3619: Improved simple language with syntax parser and ast.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BaseSimpleParser.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BinaryOperatorType.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/LogicalOperatorType.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleBackwardsCompatibleParser.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleExpressionParser.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleIllegalSyntaxException.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleParserException.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimplePredicateParser.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleToken.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleTokenType.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleTokenizer.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/TokenType.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/UnaryOperatorType.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/BaseSimpleNode.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/BinaryExpression.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/Block.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/BlockEnd.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/BlockStart.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/CompositeNodes.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/DoubleQuoteEnd.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/DoubleQuoteStart.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/LiteralExpression.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/LiteralNode.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/LogicalExpression.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/NullExpression.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/SimpleFunctionEnd.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/SimpleFunctionExpression.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/SimpleFunctionStart.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/SimpleNode.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/SingleQuoteEnd.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/SingleQuoteStart.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/UnaryExpression.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/ast/package.html
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleBackwardsCompatibleTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleChangeFunctionTokensTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleOperatorTest.java   (contents, props changed)
      - copied, changed from r1167201, camel/trunk/camel-core/src/test/java/org/apache/camel/language/SimpleOperatorTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleParserExpressionInvalidTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleParserExpressionTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleParserPredicateInvalidTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleParserPredicateTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleParserRegexpPredicateTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/simple/SimpleTest.java   (contents, props changed)
      - copied, changed from r1167201, camel/trunk/camel-core/src/test/java/org/apache/camel/language/SimpleTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/builder/PredicateBuilder.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleLanguage.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleLanguageOperator.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleLanguageSupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/package.html
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanParameterInvalidValueTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/FileLanguageTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/SimpleOperatorTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/language/SimpleTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/builder/PredicateBuilder.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/PredicateBuilder.java?rev=1167462&r1=1167461&r2=1167462&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/builder/PredicateBuilder.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/builder/PredicateBuilder.java Sat Sep 10 05:43:14 2011
@@ -17,6 +17,7 @@
 package org.apache.camel.builder;
 
 import java.util.Arrays;
+import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -434,4 +435,23 @@ public final class PredicateBuilder {
             }
         };
     }
+
+    /**
+     * Concat the given predicates into a single predicate, which
+     * only matches if all the predicates matches.
+     *
+     * @param predicates predicates
+     * @return a single predicate containing all the predicates
+     */
+    public static Predicate and(List<Predicate> predicates) {
+        Predicate answer = null;
+        for (Predicate predicate : predicates) {
+            if (answer == null) {
+                answer = predicate;
+            } else {
+                answer = and(answer, predicate);
+            }
+        }
+        return answer;
+    }
 }

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BaseSimpleParser.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BaseSimpleParser.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BaseSimpleParser.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BaseSimpleParser.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Stack;
+
+import org.apache.camel.language.simple.ast.Block;
+import org.apache.camel.language.simple.ast.BlockEnd;
+import org.apache.camel.language.simple.ast.BlockStart;
+import org.apache.camel.language.simple.ast.SimpleNode;
+import org.apache.camel.language.simple.ast.UnaryExpression;
+
+/**
+ * Base class for Simple language parser.
+ * <p/>
+ * This parser is based on the principles of a
+ * <a href="http://en.wikipedia.org/wiki/Recursive_descent_parser">recursive descent parser</a>.
+ */
+public abstract class BaseSimpleParser {
+
+    protected final String expression;
+    protected final List<SimpleToken> tokens = new ArrayList<SimpleToken>();
+    protected final List<SimpleNode> nodes = new ArrayList<SimpleNode>();
+    protected SimpleToken token;
+    protected int previousIndex;
+    protected int index;
+
+    protected BaseSimpleParser(String expression) {
+        this.expression = expression;
+    }
+
+    /**
+     * Advances the parser position to the next known {@link SimpleToken}
+     * in the input.
+     */
+    protected void nextToken() {
+        if (index < expression.length()) {
+            SimpleToken next = SimpleTokenizer.nextToken(expression, index);
+            // add token
+            tokens.add(next);
+            token = next;
+            // position index after the token
+            previousIndex = index;
+            index += next.getLength();
+        } else {
+            // end of tokens
+            token = new SimpleToken(new SimpleTokenType(TokenType.eol, null), index);
+        }
+    }
+
+    /**
+     * Advances the parser position to the next known {@link SimpleToken}
+     * in the input.
+     *
+     * @param filter filter for accepted token types
+     */
+    protected void nextToken(TokenType... filter) {
+        if (index < expression.length()) {
+            SimpleToken next = SimpleTokenizer.nextToken(expression, index, filter);
+            // add token
+            tokens.add(next);
+            token = next;
+            // position index after the token
+            previousIndex = index;
+            index += next.getLength();
+        } else {
+            // end of tokens
+            token = new SimpleToken(new SimpleTokenType(TokenType.eol, null), index);
+        }
+    }
+
+    /**
+     * Clears the parser state, which means it can be used for parsing a new input.
+     */
+    protected void clear() {
+        token = null;
+        previousIndex = 0;
+        index = 0;
+        tokens.clear();
+        nodes.clear();
+    }
+
+    /**
+     * Prepares blocks, such as functions, single or double quoted texts.
+     * <p/>
+     * This process prepares the {@link Block}s in the AST. This is done
+     * by linking child {@link SimpleNode nodes} which are within the start and end of the blocks,
+     * as child to the given block. This is done to have the AST graph updated and prepared properly.
+     * <p/>
+     * So when the AST node is later used to create the {@link org.apache.camel.Predicate}s
+     * to be used by Camel then the AST graph has a linked and prepared
+     * graph of nodes which represent the input expression.
+     */
+    protected void prepareBlocks() {
+        List<SimpleNode> answer = new ArrayList<SimpleNode>();
+        Stack<Block> stack = new Stack<Block>();
+
+        for (SimpleNode token : nodes) {
+            if (token instanceof BlockStart) {
+                // a new block is started, so push on the stack
+                stack.push((Block) token);
+            } else if (token instanceof BlockEnd) {
+                // end block is just an abstract mode, so we should not add it
+                if (stack.isEmpty()) {
+                    throw new SimpleParserException(token.getToken().getType().getType() + " has no matching start token", token.getToken().getIndex());
+                }
+                Block top = stack.pop();
+                answer.add(top);
+            } else {
+                // if there is a model on the stack then it should accept the child model
+                Block block = stack.isEmpty() ? null : stack.peek();
+                if (block != null) {
+                    if (!block.acceptAndAddNode(token)) {
+                        throw new SimpleParserException(block.getToken().getType() + " cannot accept " + token.getToken().getType(), token.getToken().getIndex());
+                    }
+                } else {
+                    // no block, so add to answer
+                    answer.add(token);
+                }
+            }
+        }
+
+        // replace nodes from the stack
+        nodes.clear();
+        nodes.addAll(answer);
+    }
+
+    /**
+     * Prepares unary expressions.
+     * <p/>
+     * This process prepares the unary expressions in the AST. This is done
+     * by linking the unary operator with the left hand side node,
+     * to have the AST graph updated and prepared properly.
+     * <p/>
+     * So when the AST node is later used to create the {@link org.apache.camel.Predicate}s
+     * to be used by Camel then the AST graph has a linked and prepared
+     * graph of nodes which represent the input expression.
+     */
+    protected void prepareUnaryExpressions() {
+        Stack<SimpleNode> stack = new Stack<SimpleNode>();
+
+        for (SimpleNode node : nodes) {
+            if (node instanceof UnaryExpression) {
+                UnaryExpression token = (UnaryExpression) node;
+
+                // remember the logical operator
+                String operator = token.getOperator().toString();
+
+                SimpleNode previous = stack.isEmpty() ? null : stack.pop();
+                if (previous == null) {
+                    throw new SimpleParserException("Unary operator " + operator + " has no left hand side token", token.getToken().getIndex());
+                } else {
+                    token.acceptLeft(previous);
+                }
+            }
+            stack.push(node);
+        }
+
+        // replace nodes from the stack
+        nodes.clear();
+        nodes.addAll(stack);
+    }
+
+    // --------------------------------------------------------------
+    // grammar
+    // --------------------------------------------------------------
+
+    /**
+     * Accept the given token.
+     * <p/>
+     * This is to be used by the grammar to accept tokens and then continue parsing
+     * using the grammar, such as a function grammar.
+     *
+     * @param accept  the token
+     * @return <tt>true</tt> if accepted, <tt>false</tt> otherwise.
+     */
+    protected boolean accept(TokenType accept) {
+        return token == null || token.getType().getType() == accept;
+    }
+
+    /**
+     * Expect a given token
+     *
+     * @param expect the token to expect
+     * @throws SimpleParserException is thrown if the token is not as expected
+     */
+    protected void expect(TokenType expect) throws SimpleParserException {
+        if (token != null && token.getType().getType() == expect) {
+            return;
+        } else if (token == null) {
+            // use the previous index as that is where the problem is
+            throw new SimpleParserException("expected symbol " + expect + " but reached eol", previousIndex);
+        } else {
+            // use the previous index as that is where the problem is
+            throw new SimpleParserException("expected symbol " + expect + " but was " + token.getType().getType(), previousIndex);
+        }
+    }
+
+    /**
+     * Expect and accept a given number of tokens in sequence.
+     * <p/>
+     * This is used to accept whitespace or string literals.
+     *
+     * @param expect the token to accept
+     */
+    protected void expectAndAcceptMore(TokenType expect) {
+        expect(expect);
+
+        while (!token.getType().isEol() && token.getType().getType() == expect) {
+            nextToken();
+        }
+    }
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BinaryOperatorType.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BinaryOperatorType.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BinaryOperatorType.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/BinaryOperatorType.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+/**
+ * Types of binary operators supported
+ */
+public enum BinaryOperatorType {
+
+    EQ, GT, GTE, LT, LTE, NOT_EQ, CONTAINS, NOT_CONTAINS, REGEX, NOT_REGEX,
+    IN, NOT_IN, IS, NOT_IS, RANGE, NOT_RANGE;
+
+    public static BinaryOperatorType asOperator(String text) {
+        if ("==".equals(text)) {
+            return EQ;
+        } else if (">".equals(text)) {
+            return GT;
+        } else if (">=".equals(text)) {
+            return GTE;
+        } else if ("<".equals(text)) {
+            return LT;
+        } else if ("<=".equals(text)) {
+            return LTE;
+        } else if ("!=".equals(text)) {
+            return NOT_EQ;
+        } else if ("contains".equals(text)) {
+            return CONTAINS;
+        } else if ("not contains".equals(text)) {
+            return NOT_CONTAINS;
+        } else if ("regex".equals(text)) {
+            return REGEX;
+        } else if ("not regex".equals(text)) {
+            return NOT_REGEX;
+        } else if ("in".equals(text)) {
+            return IN;
+        } else if ("not in".equals(text)) {
+            return NOT_IN;
+        } else if ("is".equals(text)) {
+            return IS;
+        } else if ("not is".equals(text)) {
+            return NOT_IS;
+        } else if ("range".equals(text)) {
+            return RANGE;
+        } else if ("not range".equals(text)) {
+            return NOT_RANGE;
+        }
+        throw new IllegalArgumentException("Operator not supported: " + text);
+    }
+
+    public static String getOperatorText(BinaryOperatorType operator) {
+        if (operator == EQ) {
+            return "==";
+        } else if (operator == GT) {
+            return ">";
+        } else if (operator == GTE) {
+            return ">=";
+        } else if (operator == LT) {
+            return "<";
+        } else if (operator == LTE) {
+            return "<=";
+        } else if (operator == NOT_EQ) {
+            return "!=";
+        } else if (operator == CONTAINS) {
+            return "contains";
+        } else if (operator == NOT_CONTAINS) {
+            return "not contains";
+        } else if (operator == REGEX) {
+            return "regex";
+        } else if (operator == NOT_REGEX) {
+            return "not regex";
+        } else if (operator == IN) {
+            return "in";
+        } else if (operator == NOT_IN) {
+            return "not in";
+        } else if (operator == IS) {
+            return "is";
+        } else if (operator == NOT_IS) {
+            return "not is";
+        } else if (operator == RANGE) {
+            return "range";
+        } else if (operator == NOT_RANGE) {
+            return "not range";
+        }
+        return "";
+    }
+
+    /**
+     * Parameter types a binary operator supports on the right hand side.
+     * <ul>
+     *     <li>Literal - Only literals enclosed by single quotes</li>
+     *     <li>LiteralWithFunction - literals which may have embedded functions enclosed by single quotes</li>
+     *     <li>Function - A function</li>
+     *     <li>NumericValue - A numeric value</li>
+     *     <li>BooleanValue - A boolean value</li>
+     *     <li>NullValue - A null value</li>
+     * </ul>
+     */
+    public enum ParameterType {
+        Literal, LiteralWithFunction, Function, NumericValue, BooleanValue, NullValue;
+
+        boolean isLiteralSupported() {
+            return this == Literal;
+        }
+
+        boolean isLiteralWithFunctionSupport() {
+            return this == LiteralWithFunction;
+        }
+
+        boolean isFunctionSupport() {
+            return this == Function;
+        }
+
+        boolean isNumericValueSupported() {
+            return this == NumericValue;
+        }
+
+        boolean isBooleanValueSupported() {
+            return this == BooleanValue;
+        }
+
+        boolean isNullValueSupported() {
+            return this == NullValue;
+        }
+    }
+
+    /**
+     * Returns the types of right hand side parameters this operator supports.
+     *
+     * @param operator the operator
+     * @return <tt>null</tt> if accepting all types, otherwise the array of accepted types
+     */
+    public static ParameterType[] supportedParameterTypes(BinaryOperatorType operator) {
+        if (operator == EQ) {
+            return null;
+        } else if (operator == GT) {
+            return null;
+        } else if (operator == GTE) {
+            return null;
+        } else if (operator == LT) {
+            return null;
+        } else if (operator == LTE) {
+            return null;
+        } else if (operator == NOT_EQ) {
+            return null;
+        } else if (operator == CONTAINS) {
+            return null;
+        } else if (operator == NOT_CONTAINS) {
+            return null;
+        } else if (operator == REGEX) {
+            return new ParameterType[]{ParameterType.Literal, ParameterType.Function};
+        } else if (operator == NOT_REGEX) {
+            return new ParameterType[]{ParameterType.Literal, ParameterType.Function};
+        } else if (operator == IN) {
+            return null;
+        } else if (operator == NOT_IN) {
+            return null;
+        } else if (operator == IS) {
+            return new ParameterType[]{ParameterType.LiteralWithFunction, ParameterType.Function};
+        } else if (operator == NOT_IS) {
+            return new ParameterType[]{ParameterType.LiteralWithFunction, ParameterType.Function};
+        } else if (operator == RANGE) {
+            return new ParameterType[]{ParameterType.LiteralWithFunction, ParameterType.Function};
+        } else if (operator == NOT_RANGE) {
+            return new ParameterType[]{ParameterType.LiteralWithFunction, ParameterType.Function};
+        }
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return getOperatorText(this);
+    }
+
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/LogicalOperatorType.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/LogicalOperatorType.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/LogicalOperatorType.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/LogicalOperatorType.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+/**
+ * Types of logical operators supported
+ */
+public enum LogicalOperatorType {
+
+    // TODO: and|or is @deprecated and to be removed in Camel 3.0
+
+    AND, OR;
+
+    public static LogicalOperatorType asOperator(String text) {
+        if ("&&".equals(text) || "and".equals(text)) {
+            return AND;
+        } else if ("||".equals(text) || "or".equals(text)) {
+            return OR;
+        }
+        throw new IllegalArgumentException("Operator not supported: " + text);
+    }
+
+    public String getOperatorText(LogicalOperatorType operator) {
+        if (operator == AND) {
+            return "&&";
+        } else if (operator == OR) {
+            return "||";
+        }
+        return "";
+    }
+
+    @Override
+    public String toString() {
+        return getOperatorText(this);
+    }
+
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleBackwardsCompatibleParser.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleBackwardsCompatibleParser.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleBackwardsCompatibleParser.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleBackwardsCompatibleParser.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+import org.apache.camel.Expression;
+import org.apache.camel.Predicate;
+import org.apache.camel.builder.PredicateBuilder;
+import org.apache.camel.language.simple.ast.SimpleFunctionExpression;
+
+/**
+ * A backwards compatible parser, which supports the old simple language
+ * syntax by which simple functions can be given without using start and
+ * end tokens.
+ * <p/>
+ * For example "body" would be parsed as the body function, where as the
+ * new parser would require that to be entered as "${body}".
+ * <p/>
+ * This parser is to be removed when the old syntax is no longer supported.
+ *
+ * @deprecated will be removed in Camel 3.0
+ */
+@Deprecated
+public final class SimpleBackwardsCompatibleParser {
+
+    private SimpleBackwardsCompatibleParser() {
+        // static methods
+    }
+
+    public static Expression parseExpression(String expression) {
+        return doParseExpression(expression);
+    }
+
+    public static Predicate parsePredicate(String expression) {
+        Expression answer = doParseExpression(expression);
+        if (answer != null) {
+            return PredicateBuilder.toPredicate(answer);
+        } else {
+            return null;
+        }
+    }
+
+    private static Expression doParseExpression(String expression) {
+        // should have no function tokens
+        for (int i = 0; i < expression.length(); i++) {
+            SimpleToken token = SimpleTokenizer.nextToken(expression, i, TokenType.functionStart, TokenType.functionEnd);
+            if (token.getType().getType() == TokenType.functionStart || token.getType().getType() == TokenType.functionEnd) {
+                return null;
+            }
+        }
+
+        // okay there is no function tokens, then try to parse it as a simple function expression
+        SimpleToken token = new SimpleToken(new SimpleTokenType(TokenType.functionStart, expression), 0);
+        SimpleFunctionExpression function = new SimpleFunctionExpression(token);
+        function.addText(expression);
+        return function.createExpression(expression, false);
+    }
+
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleExpressionParser.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleExpressionParser.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleExpressionParser.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleExpressionParser.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.camel.Expression;
+import org.apache.camel.builder.ExpressionBuilder;
+import org.apache.camel.language.simple.ast.LiteralExpression;
+import org.apache.camel.language.simple.ast.LiteralNode;
+import org.apache.camel.language.simple.ast.SimpleFunctionEnd;
+import org.apache.camel.language.simple.ast.SimpleFunctionStart;
+import org.apache.camel.language.simple.ast.SimpleNode;
+import org.apache.camel.language.simple.ast.UnaryExpression;
+
+/**
+ * A parser to parse simple language as a Camel {@link Expression}
+ */
+public class SimpleExpressionParser extends BaseSimpleParser {
+
+    public SimpleExpressionParser(String expression) {
+        super(expression);
+    }
+
+    public Expression parseExpression() {
+        clear();
+        try {
+            return doParseExpression();
+        } catch (SimpleParserException e) {
+            // catch parser exception and turn that into a syntax exceptions
+            throw new SimpleIllegalSyntaxException(expression, e.getIndex(), e.getMessage(), e);
+        } catch (Exception e) {
+            // include exception in rethrown exception
+            throw new SimpleIllegalSyntaxException(expression, -1, e.getMessage(), e);
+        }
+    }
+
+    protected Expression doParseExpression() {
+        // parse the expression using the following grammar
+        nextToken();
+        while (!token.getType().isEol()) {
+            // an expression supports just template (eg text), functions, or unary operator
+            templateText();
+            functionText();
+            unaryOperator();
+            nextToken();
+        }
+
+        // now after parsing we need a bit of work to do, to make it easier to turn the tokens
+        // into and ast, and then from the ast, to Camel expression(s).
+        // hence why there is a number of tasks going on below to accomplish this
+
+        // turn the tokens into the ast model
+        parseAndCreateAstModel();
+        // compact and stack blocks (eg function start/end)
+        prepareBlocks();
+        // compact and stack unary operators
+        prepareUnaryExpressions();
+
+        // create and return as a Camel expression
+        List<Expression> expressions = createExpressions();
+        if (expressions.isEmpty()) {
+            return null;
+        } else if (expressions.size() == 1) {
+            return expressions.get(0);
+        } else {
+            // concat expressions as evaluating an expression is like a template language
+            return ExpressionBuilder.concatExpression(expressions, expression);
+        }
+    }
+
+    protected void parseAndCreateAstModel() {
+        // we loop the tokens and create a sequence of ast nodes
+
+        LiteralNode imageToken = null;
+        for (SimpleToken token : tokens) {
+            // break if eol
+            if (token.getType().isEol()) {
+                break;
+            }
+
+            // create a node from the token
+            SimpleNode node = createNode(token);
+            if (node != null) {
+                // a new token was created so the current image token need to be added first
+                if (imageToken != null) {
+                    nodes.add(imageToken);
+                    imageToken = null;
+                }
+                // and then add the created node
+                nodes.add(node);
+                // continue to next
+                continue;
+            }
+
+            // if no token was created then its a character/whitespace/escaped symbol
+            // which we need to add together in the same image
+            if (imageToken == null) {
+                imageToken = new LiteralExpression(token);
+            }
+            imageToken.addText(token.getText());
+        }
+
+        // append any leftover image tokens (when we reached eol)
+        if (imageToken != null) {
+            nodes.add(imageToken);
+        }
+    }
+
+    private SimpleNode createNode(SimpleToken token) {
+        // expression only support functions and unary operators
+        if (token.getType().isFunctionStart()) {
+            return new SimpleFunctionStart(token);
+        } else if (token.getType().isFunctionEnd()) {
+            return new SimpleFunctionEnd(token);
+        } else if (token.getType().isUnary()) {
+            return new UnaryExpression(token);
+        }
+
+        // by returning null, we will let the parser determine what to do
+        return null;
+    }
+
+    private List<Expression> createExpressions() {
+        List<Expression> answer = new ArrayList<Expression>();
+        for (SimpleNode token : nodes) {
+            Expression exp = token.createExpression(expression);
+            if (exp != null) {
+                answer.add(exp);
+            }
+        }
+        return answer;
+    }
+
+    // --------------------------------------------------------------
+    // grammar
+    // --------------------------------------------------------------
+
+    // the expression parser only understands
+    // - template = literal texts with can contain embedded functions
+    // - function = simple functions such as ${body} etc
+    // - unary operator = operator attached to the left hand side node
+
+    protected void templateText() {
+        // for template we accept anything but functions
+        while (!token.getType().isFunctionStart() && !token.getType().isFunctionEnd() && !token.getType().isEol()) {
+            nextToken();
+        }
+    }
+
+    protected boolean functionText() {
+        if (accept(TokenType.functionStart)) {
+            nextToken();
+            while (!token.getType().isFunctionEnd() && !token.getType().isEol()) {
+                // we need to loop until we find the ending function quote, or the eol
+                nextToken();
+            }
+            expect(TokenType.functionEnd);
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean unaryOperator() {
+        if (accept(TokenType.unaryOperator)) {
+            nextToken();
+            // there should be a whitespace after the operator
+            expect(TokenType.whiteSpace);
+            return true;
+        }
+        return false;
+    }
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleIllegalSyntaxException.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleIllegalSyntaxException.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleIllegalSyntaxException.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleIllegalSyntaxException.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+import org.apache.camel.ExpressionIllegalSyntaxException;
+
+/**
+ * Syntax error in the simple language expression.
+ */
+public class SimpleIllegalSyntaxException extends ExpressionIllegalSyntaxException {
+
+    private final int index;
+    private final String message;
+
+    public SimpleIllegalSyntaxException(String expression, int index, String message) {
+        super(expression);
+        this.index = index;
+        this.message = message;
+    }
+
+    public SimpleIllegalSyntaxException(String expression, int index, String message, Throwable cause) {
+        super(expression, cause);
+        this.index = index;
+        this.message = message;
+    }
+
+    /**
+     * Index where the parsing error occurred
+     *
+     * @return index of the parsing error in the input, returns <tt>-1</tt> if the cause of the problem
+     * is not applicable to specific index in the input
+     */
+    public int getIndex() {
+        return index;
+    }
+
+    @Override
+    public String getMessage() {
+        if (message == null) {
+            return "[null]";
+        }
+
+        StringBuilder sb = new StringBuilder(message);
+        if (index > -1) {
+            sb.append(" at location ").append(index);
+            // create a nice looking message with indicator where the problem is
+            sb.append("\n").append(getExpression()).append("\n");
+            for (int i = 0; i < index; i++) {
+                sb.append(" ");
+            }
+            sb.append("*\n");
+        }
+        return sb.toString();
+    }
+
+}

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleLanguage.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleLanguage.java?rev=1167462&r1=1167461&r2=1167462&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleLanguage.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleLanguage.java Sat Sep 10 05:43:14 2011
@@ -17,11 +17,10 @@
 package org.apache.camel.language.simple;
 
 import org.apache.camel.Expression;
-import org.apache.camel.ExpressionIllegalSyntaxException;
+import org.apache.camel.IsSingleton;
+import org.apache.camel.Predicate;
 import org.apache.camel.builder.ExpressionBuilder;
-import org.apache.camel.util.ObjectHelper;
-import org.apache.camel.util.OgnlHelper;
-import org.apache.camel.util.StringHelper;
+import org.apache.camel.spi.Language;
 
 /**
  * A <a href="http://camel.apache.org/simple.html">simple language</a>
@@ -87,9 +86,8 @@ import org.apache.camel.util.StringHelpe
  * <br/>
  * The <b>only</b> file is the filename only with all paths clipped.
  *
- * @version
  */
-public class SimpleLanguage extends SimpleLanguageSupport {
+public class SimpleLanguage implements Language, IsSingleton {
 
     // singleton for expressions without a result type
     private static final SimpleLanguage SIMPLE = new SimpleLanguage();
@@ -110,11 +108,27 @@ public class SimpleLanguage extends Simp
         return false;
     }
 
-    @Override
+    public Predicate createPredicate(String expression) {
+        // support old simple language syntax
+        Predicate answer = SimpleBackwardsCompatibleParser.parsePredicate(expression);
+        if (answer == null) {
+            // use the new parser
+            SimplePredicateParser parser = new SimplePredicateParser(expression);
+            answer = parser.parsePredicate();
+        }
+        return answer;
+    }
+
     public Expression createExpression(String expression) {
-        Expression answer = super.createExpression(expression);
+        // support old simple language syntax
+        Expression answer = SimpleBackwardsCompatibleParser.parseExpression(expression);
+        if (answer == null) {
+            // use the new parser
+            SimpleExpressionParser parser = new SimpleExpressionParser(expression);
+            answer = parser.parseExpression();
+        }
         if (resultType != null) {
-            return ExpressionBuilder.convertToExpression(answer, resultType);
+            answer = ExpressionBuilder.convertToExpression(answer, resultType);
         }
         return answer;
     }
@@ -129,260 +143,12 @@ public class SimpleLanguage extends Simp
         return answer.createExpression(expression);
     }
 
-    protected Expression createSimpleExpressionDirectly(String expression) {
-
-        if (ObjectHelper.isEqualToAny(expression, "body", "in.body")) {
-            return ExpressionBuilder.bodyExpression();
-        } else if (ObjectHelper.equal(expression, "out.body")) {
-            return ExpressionBuilder.outBodyExpression();
-        } else if (ObjectHelper.equal(expression, "id")) {
-            return ExpressionBuilder.messageIdExpression();
-        } else if (ObjectHelper.equal(expression, "exchangeId")) {
-            return ExpressionBuilder.exchangeIdExpression();
-        } else if (ObjectHelper.equal(expression, "exception")) {
-            return ExpressionBuilder.exchangeExceptionExpression();
-        } else if (ObjectHelper.equal(expression, "exception.message")) {
-            return ExpressionBuilder.exchangeExceptionMessageExpression();
-        } else if (ObjectHelper.equal(expression, "exception.stacktrace")) {
-            return ExpressionBuilder.exchangeExceptionStackTraceExpression();
-        } else if (ObjectHelper.equal(expression, "threadName")) {
-            return ExpressionBuilder.threadNameExpression();
-        }
-
-        return null;
+    public static void changeFunctionStartToken(String... startToken) {
+        SimpleTokenizer.changeFunctionStartToken(startToken);
     }
-
-    protected Expression createSimpleExpression(String expression, boolean strict) {
-
-        // return the expression directly if we can create expression without analyzing the prefix
-        Expression answer = createSimpleExpressionDirectly(expression);
-        if (answer != null) {
-            return answer;
-        }
-
-        // bodyAs
-        String remainder = ifStartsWithReturnRemainder("bodyAs", expression);
-        if (remainder != null) {
-            String type = ObjectHelper.between(remainder, "(", ")");
-            if (type == null) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${bodyAs(type)} was: " + expression);
-            }
-            type = StringHelper.removeQuotes(type);
-            return ExpressionBuilder.bodyExpression(type);
-        }
-        // mandatoryBodyAs
-        remainder = ifStartsWithReturnRemainder("mandatoryBodyAs", expression);
-        if (remainder != null) {
-            String type = ObjectHelper.between(remainder, "(", ")");
-            if (type == null) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${mandatoryBodyAs(type)} was: " + expression);
-            }
-            type = StringHelper.removeQuotes(type);
-            return ExpressionBuilder.mandatoryBodyExpression(type);
-        }
-
-        // body OGNL
-        remainder = ifStartsWithReturnRemainder("body", expression);
-        if (remainder == null) {
-            remainder = ifStartsWithReturnRemainder("in.body", expression);
-        }
-        if (remainder != null) {
-            boolean invalid = OgnlHelper.isInvalidValidOgnlExpression(remainder);
-            if (invalid) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${body.OGNL} was: " + expression);
-            }
-            return ExpressionBuilder.bodyOgnlExpression(remainder);
-        }
-
-        // Exception OGNL
-        remainder = ifStartsWithReturnRemainder("exception", expression);
-        if (remainder != null) {
-            boolean invalid = OgnlHelper.isInvalidValidOgnlExpression(remainder);
-            if (invalid) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${exception.OGNL} was: " + expression);
-            }
-            return ExpressionBuilder.exchangeExceptionOgnlExpression(remainder);
-        }
-
-        // headerAs
-        remainder = ifStartsWithReturnRemainder("headerAs", expression);
-        if (remainder != null) {
-            String keyAndType = ObjectHelper.between(remainder, "(", ")");
-            if (keyAndType == null) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${headerAs(key, type)} was: " + expression);
-            }
-
-            String key = ObjectHelper.before(keyAndType, ",");
-            String type = ObjectHelper.after(keyAndType, ",");
-            if (ObjectHelper.isEmpty(key) || ObjectHelper.isEmpty(type)) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${headerAs(key, type)} was: " + expression);
-            }
-            key = StringHelper.removeQuotes(key);
-            type = StringHelper.removeQuotes(type);
-            return ExpressionBuilder.headerExpression(key, type);
-        }
-
-        // headers expression
-        if ("in.headers".equals(expression) || "headers".equals(expression)) {
-            return ExpressionBuilder.headersExpression();
-        }
-
-        // in header expression
-        remainder = ifStartsWithReturnRemainder("in.headers", expression);
-        if (remainder == null) {
-            remainder = ifStartsWithReturnRemainder("in.header", expression);
-        }
-        if (remainder == null) {
-            remainder = ifStartsWithReturnRemainder("headers", expression);
-        }
-        if (remainder == null) {
-            remainder = ifStartsWithReturnRemainder("header", expression);
-        }
-        if (remainder != null) {
-            // remove leading character (dot or ?)
-            remainder = remainder.substring(1);
-
-            // validate syntax
-            boolean invalid = OgnlHelper.isInvalidValidOgnlExpression(remainder);
-            if (invalid) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${header.name[key]} was: " + expression);
-            }
-
-            if (OgnlHelper.isValidOgnlExpression(remainder)) {
-                // ognl based header
-                return ExpressionBuilder.headersOgnlExpression(remainder);
-            } else {
-                // regular header
-                return ExpressionBuilder.headerExpression(remainder);
-            }
-        }
-
-        // out header expression
-        remainder = ifStartsWithReturnRemainder("out.header.", expression);
-        if (remainder == null) {
-            remainder = ifStartsWithReturnRemainder("out.headers.", expression);
-        }
-        if (remainder != null) {
-            return ExpressionBuilder.outHeaderExpression(remainder);
-        }
-
-        // property
-        remainder = ifStartsWithReturnRemainder("property", expression);
-        if (remainder != null) {
-            // remove leading character (dot or ?)
-            remainder = remainder.substring(1);
-
-            // validate syntax
-            boolean invalid = OgnlHelper.isInvalidValidOgnlExpression(remainder);
-            if (invalid) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${property.OGNL} was: " + expression);
-            }
-
-            if (OgnlHelper.isValidOgnlExpression(remainder)) {
-                // ognl based property
-                return ExpressionBuilder.propertyOgnlExpression(remainder);
-            } else {
-                // regular property
-                return ExpressionBuilder.propertyExpression(remainder);
-            }
-        }
-
-        // system property
-        remainder = ifStartsWithReturnRemainder("sys.", expression);
-        if (remainder != null) {
-            return ExpressionBuilder.systemPropertyExpression(remainder);
-        }
-
-        // system property
-        remainder = ifStartsWithReturnRemainder("sysenv.", expression);
-        if (remainder != null) {
-            return ExpressionBuilder.systemEnvironmentExpression(remainder);
-        }
-
-        // file: prefix
-        remainder = ifStartsWithReturnRemainder("file:", expression);
-        if (remainder != null) {
-            Expression fileExpression = createSimpleFileExpression(remainder);
-            if (expression != null) {
-                return fileExpression;
-            }
-        }
-
-        // date: prefix
-        remainder = ifStartsWithReturnRemainder("date:", expression);
-        if (remainder != null) {
-            String[] parts = remainder.split(":");
-            if (parts.length < 2) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${date:command:pattern} was: " + expression);
-            }
-            String command = ObjectHelper.before(remainder, ":");
-            String pattern = ObjectHelper.after(remainder, ":");
-            return ExpressionBuilder.dateExpression(command, pattern);
-        }
-
-        // bean: prefix
-        remainder = ifStartsWithReturnRemainder("bean:", expression);
-        if (remainder != null) {
-            return ExpressionBuilder.beanExpression(remainder);
-        }
-
-        // properties: prefix
-        remainder = ifStartsWithReturnRemainder("properties:", expression);
-        if (remainder != null) {
-            String[] parts = remainder.split(":");
-            if (parts.length > 2) {
-                throw new ExpressionIllegalSyntaxException("Valid syntax: ${properties:[locations]:key} was: " + expression);
-            }
-
-            String locations = null;
-            String key = remainder;
-            if (parts.length == 2) {
-                locations = ObjectHelper.before(remainder, ":");
-                key = ObjectHelper.after(remainder, ":");
-            }
-            return ExpressionBuilder.propertiesComponentExpression(key, locations);
-        }
-
-        // ref: prefix
-        remainder = ifStartsWithReturnRemainder("ref:", expression);
-        if (remainder != null) {
-            return ExpressionBuilder.refExpression(remainder);
-        }
-
-        if (strict) {
-            throw new ExpressionIllegalSyntaxException(expression);
-        } else {
-            return ExpressionBuilder.constantExpression(expression);
-        }
-    }
-
-    protected Expression createSimpleFileExpression(String remainder) {
-        if (ObjectHelper.equal(remainder, "name")) {
-            return ExpressionBuilder.fileNameExpression();
-        } else if (ObjectHelper.equal(remainder, "name.noext")) {
-            return ExpressionBuilder.fileNameNoExtensionExpression();
-        } else if (ObjectHelper.equal(remainder, "name.ext")) {
-            return ExpressionBuilder.fileExtensionExpression();
-        } else if (ObjectHelper.equal(remainder, "onlyname")) {
-            return ExpressionBuilder.fileOnlyNameExpression();
-        } else if (ObjectHelper.equal(remainder, "onlyname.noext")) {
-            return ExpressionBuilder.fileOnlyNameNoExtensionExpression();
-        } else if (ObjectHelper.equal(remainder, "ext")) {
-            return ExpressionBuilder.fileExtensionExpression();
-        } else if (ObjectHelper.equal(remainder, "parent")) {
-            return ExpressionBuilder.fileParentExpression();
-        } else if (ObjectHelper.equal(remainder, "path")) {
-            return ExpressionBuilder.filePathExpression();
-        } else if (ObjectHelper.equal(remainder, "absolute")) {
-            return ExpressionBuilder.fileAbsoluteExpression();
-        } else if (ObjectHelper.equal(remainder, "absolute.path")) {
-            return ExpressionBuilder.fileAbsolutePathExpression();
-        } else if (ObjectHelper.equal(remainder, "length") || ObjectHelper.equal(remainder, "size")) {
-            return ExpressionBuilder.fileSizeExpression();
-        } else if (ObjectHelper.equal(remainder, "modified")) {
-            return ExpressionBuilder.fileLastModifiedExpression();
-        }
-        throw new ExpressionIllegalSyntaxException("File language syntax: " + remainder);
+    
+    public static void changeFunctionEndToken(String... endToken) {
+        SimpleTokenizer.changeFunctionEndToken(endToken);
     }
 
 }

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleParserException.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleParserException.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleParserException.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleParserException.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+import org.apache.camel.RuntimeCamelException;
+
+/**
+ * Holds information about error parsing the simple expression at a given location
+ * in the input.
+ */
+public class SimpleParserException extends RuntimeCamelException {
+
+    private final int index;
+
+    public SimpleParserException(String message, int index) {
+        super(message);
+        this.index = index;
+    }
+
+    /**
+     * Index where the parsing error occurred
+     *
+     * @return index of the parsing error in the input
+     */
+    public int getIndex() {
+        return index;
+    }
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimplePredicateParser.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimplePredicateParser.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimplePredicateParser.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimplePredicateParser.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,635 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Stack;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.camel.Expression;
+import org.apache.camel.Predicate;
+import org.apache.camel.builder.PredicateBuilder;
+import org.apache.camel.language.simple.ast.BinaryExpression;
+import org.apache.camel.language.simple.ast.DoubleQuoteEnd;
+import org.apache.camel.language.simple.ast.DoubleQuoteStart;
+import org.apache.camel.language.simple.ast.LiteralExpression;
+import org.apache.camel.language.simple.ast.LiteralNode;
+import org.apache.camel.language.simple.ast.LogicalExpression;
+import org.apache.camel.language.simple.ast.NullExpression;
+import org.apache.camel.language.simple.ast.SimpleFunctionEnd;
+import org.apache.camel.language.simple.ast.SimpleFunctionStart;
+import org.apache.camel.language.simple.ast.SimpleNode;
+import org.apache.camel.language.simple.ast.SingleQuoteEnd;
+import org.apache.camel.language.simple.ast.SingleQuoteStart;
+import org.apache.camel.language.simple.ast.UnaryExpression;
+
+/**
+ * A parser to parse simple language as a Camel {@link Predicate}
+ */
+public class SimplePredicateParser extends BaseSimpleParser {
+
+    public SimplePredicateParser(String expression) {
+        super(expression);
+    }
+
+    public Predicate parsePredicate() {
+        clear();
+        try {
+            return doParsePredicate();
+        } catch (SimpleParserException e) {
+            // catch parser exception and turn that into a syntax exceptions
+            throw new SimpleIllegalSyntaxException(expression, e.getIndex(), e.getMessage(), e);
+        } catch (Exception e) {
+            // include exception in rethrown exception
+            throw new SimpleIllegalSyntaxException(expression, -1, e.getMessage(), e);
+        }
+    }
+
+    protected Predicate doParsePredicate() {
+
+        // parse using the following grammar
+        nextToken();
+        while (!token.getType().isEol()) {
+            // predicate supports quotes, functions, operators and whitespaces
+            //CHECKSTYLE:OFF
+            if (!singleQuotedLiteralWithFunctionsText()
+                    && !doubleQuotedLiteralWithFunctionsText()
+                    && !functionText()
+                    && !unaryOperator()
+                    && !binaryOperator()
+                    && !logicalOperator()
+                    && !token.getType().isWhitespace()
+                    && !token.getType().isEol()) {
+                // okay the symbol was not one of the above, so its not supported
+                // use the previous index as that is where the problem is
+                throw new SimpleParserException("Unexpected token " + token, previousIndex);
+            }
+            //CHECKSTYLE:ON
+            // take the next token
+            nextToken();
+        }
+
+        // now after parsing we need a bit of work to do, to make it easier to turn the tokens
+        // into and ast, and then from the ast, to Camel predicate(s).
+        // hence why there is a number of tasks going on below to accomplish this
+
+        // remove any ignorable white space tokens
+        removeIgnorableWhiteSpaceTokens();
+        // turn the tokens into the ast model
+        parseTokensAndCreateNodes();
+        // compact and stack blocks (eg function start/end, quotes start/end, etc.)
+        prepareBlocks();
+        // compact and stack unary expressions
+        prepareUnaryExpressions();
+        // compact and stack binary expressions
+        prepareBinaryExpressions();
+        // compact and stack logical expressions
+        prepareLogicalExpressions();
+
+        // create and return as a Camel predicate
+        List<Predicate> predicates = createPredicates();
+        if (predicates.isEmpty()) {
+            return null;
+        } else if (predicates.size() == 1) {
+            return predicates.get(0);
+        } else {
+            return PredicateBuilder.and(predicates);
+        }
+    }
+
+    /**
+     * Parses the tokens and crates the AST nodes.
+     * <p/>
+     * After the initial parsing of the input (input -> tokens) then we
+     * parse again (tokens -> ast).
+     * <p/>
+     * In this parsing the balance of the blocks is checked, so that each block has a matching
+     * start and end token. For example a single quote block, or a function block etc.
+     */
+    protected void parseTokensAndCreateNodes() {
+        // we loop the tokens and create a sequence of ast nodes
+
+        // we need to keep a bit of state for keeping track of single and double quotes
+        // which need to be balanced and have matching start/end pairs
+        SimpleNode lastSingle = null;
+        SimpleNode lastDouble = null;
+        SimpleNode lastFunction = null;
+        AtomicBoolean startSingle = new AtomicBoolean(false);
+        AtomicBoolean startDouble = new AtomicBoolean(false);
+        AtomicBoolean startFunction = new AtomicBoolean(false);
+
+        LiteralNode imageToken = null;
+        for (SimpleToken token : tokens) {
+            // break if eol
+            if (token.getType().isEol()) {
+                break;
+            }
+
+            // create a node from the token
+            SimpleNode node = createNode(token, startSingle, startDouble, startFunction);
+            if (node != null) {
+                // keep state of last single/double
+                if (node instanceof SingleQuoteStart) {
+                    lastSingle = node;
+                } else if (node instanceof DoubleQuoteStart) {
+                    lastDouble = node;
+                } else if (node instanceof SimpleFunctionStart) {
+                    lastFunction = node;
+                }
+
+                // a new token was created so the current image token need to be added first
+                if (imageToken != null) {
+                    nodes.add(imageToken);
+                    imageToken = null;
+                }
+                // and then add the created node
+                nodes.add(node);
+                // continue to next
+                continue;
+            }
+
+            // if no token was created then its a character/whitespace/escaped symbol
+            // which we need to add together in the same image
+            if (imageToken == null) {
+                imageToken = new LiteralExpression(token);
+            }
+            imageToken.addText(token.getText());
+        }
+
+        // append any leftover image tokens (when we reached eol)
+        if (imageToken != null) {
+            nodes.add(imageToken);
+        }
+
+        // validate the single, double quote pairs and functions is in balance
+        if (startSingle.get()) {
+            int index = lastSingle != null ? lastSingle.getToken().getIndex() : 0;
+            throw new SimpleParserException("single quote has no ending quote", index);
+        }
+        if (startDouble.get()) {
+            int index = lastDouble != null ? lastDouble.getToken().getIndex() : 0;
+            throw new SimpleParserException("double quote has no ending quote", index);
+        }
+        if (startFunction.get()) {
+            // we have a start function, but no ending function
+            int index = lastFunction != null ? lastFunction.getToken().getIndex() : 0;
+            throw new SimpleParserException("function has no ending token", index);
+        }
+    }
+
+
+    /**
+     * Creates a node from the given token
+     *
+     * @param token         the token
+     * @param startSingle   state of single quoted blocks
+     * @param startDouble   state of double quoted blocks
+     * @param startFunction state of function blocks
+     * @return the created node, or <tt>null</tt> to let a default node be created instead.
+     */
+    private SimpleNode createNode(SimpleToken token, AtomicBoolean startSingle, AtomicBoolean startDouble,
+                                  AtomicBoolean startFunction) {
+        if (token.getType().isFunctionStart()) {
+            startFunction.set(true);
+            return new SimpleFunctionStart(token);
+        } else if (token.getType().isFunctionEnd()) {
+            startFunction.set(false);
+            return new SimpleFunctionEnd(token);
+        }
+
+        // if we are inside a function, then we do not support any other kind of tokens
+        // as we want all the tokens to be literal instead
+        if (startFunction.get()) {
+            return null;
+        }
+
+        // okay so far we also want to support quotes
+        if (token.getType().isSingleQuote()) {
+            SimpleNode answer;
+            boolean start = startSingle.get();
+            if (!start) {
+                answer = new SingleQuoteStart(token);
+            } else {
+                answer = new SingleQuoteEnd(token);
+            }
+            // flip state on start/end flag
+            startSingle.set(!start);
+            return answer;
+        } else if (token.getType().isDoubleQuote()) {
+            SimpleNode answer;
+            boolean start = startDouble.get();
+            if (!start) {
+                answer = new DoubleQuoteStart(token);
+            } else {
+                answer = new DoubleQuoteEnd(token);
+            }
+            // flip state on start/end flag
+            startDouble.set(!start);
+            return answer;
+        }
+
+        // if we are inside a quote, then we do not support any further kind of tokens
+        // as we want to only support embedded functions and all other kinds to be literal tokens
+        if (startSingle.get() || startDouble.get()) {
+            return null;
+        }
+
+        // okay we are not inside a function or quote, so we want to support operators
+        // and the special null value as well
+        if (token.getType().isUnary()) {
+            return new UnaryExpression(token);
+        } else if (token.getType().isBinary()) {
+            return new BinaryExpression(token);
+        } else if (token.getType().isLogical()) {
+            return new LogicalExpression(token);
+        } else if (token.getType().isNullValue()) {
+            return new NullExpression(token);
+        }
+
+        // by returning null, we will let the parser determine what to do
+        return null;
+    }
+
+    /**
+     * Removes any ignorable whitespace tokens.
+     * <p/>
+     * During the initial parsing (input -> tokens), then there may
+     * be excessive whitespace tokens, which can safely be removed,
+     * which makes the succeeding parsing easier.
+     */
+    private void removeIgnorableWhiteSpaceTokens() {
+        // white space can be removed if its not part of a quoted text
+        boolean quote = false;
+
+        Iterator<SimpleToken> it = tokens.iterator();
+        while (it.hasNext()) {
+            SimpleToken token = it.next();
+            if (token.getType().isSingleQuote()) {
+                quote = !quote;
+            } else if (token.getType().isWhitespace() && !quote) {
+                it.remove();
+            }
+        }
+    }
+
+    /**
+     * Prepares binary expressions.
+     * <p/>
+     * This process prepares the binary expressions in the AST. This is done
+     * by linking the binary operator with both the right and left hand side
+     * nodes, to have the AST graph updated and prepared properly.
+     * <p/>
+     * So when the AST node is later used to create the {@link Predicate}s
+     * to be used by Camel then the AST graph has a linked and prepared
+     * graph of nodes which represent the input expression.
+     */
+    private void prepareBinaryExpressions() {
+        Stack<SimpleNode> stack = new Stack<SimpleNode>();
+
+        SimpleNode left = null;
+        for (int i = 0; i < nodes.size(); i++) {
+            if (left == null) {
+                left = i > 0 ? nodes.get(i - 1) : null;
+            }
+            SimpleNode token = nodes.get(i);
+            SimpleNode right = i < nodes.size() - 1 ? nodes.get(i + 1) : null;
+
+            if (token instanceof BinaryExpression) {
+                BinaryExpression binary = (BinaryExpression) token;
+
+                // remember the binary operator
+                String operator = binary.getOperator().toString();
+
+                if (left == null) {
+                    throw new SimpleParserException("Binary operator " + operator + " has no left hand side token", token.getToken().getIndex());
+                }
+                if (!binary.acceptLeftNode(left)) {
+                    throw new SimpleParserException("Binary operator " + operator + " does not support left hand side token " + left.getToken(), token.getToken().getIndex());
+                }
+                if (right == null) {
+                    throw new SimpleParserException("Binary operator " + operator + " has no right hand side token", token.getToken().getIndex());
+                }
+                if (!binary.acceptRightNode(right)) {
+                    throw new SimpleParserException("Binary operator " + operator + " does not support right hand side token " + right.getToken(), token.getToken().getIndex());
+                }
+
+                // pop previous as we need to replace it with this binary operator
+                stack.pop();
+                stack.push(token);
+                // advantage after the right hand side
+                i++;
+                // this token is now the left for the next loop
+                left = token;
+            } else {
+                // clear left
+                left = null;
+                stack.push(token);
+            }
+        }
+
+        nodes.clear();
+        nodes.addAll(stack);
+    }
+
+    /**
+     * Prepares logical expressions.
+     * <p/>
+     * This process prepares the logical expressions in the AST. This is done
+     * by linking the logical operator with both the right and left hand side
+     * nodes, to have the AST graph updated and prepared properly.
+     * <p/>
+     * So when the AST node is later used to create the {@link Predicate}s
+     * to be used by Camel then the AST graph has a linked and prepared
+     * graph of nodes which represent the input expression.
+     */
+    private void prepareLogicalExpressions() {
+        Stack<SimpleNode> stack = new Stack<SimpleNode>();
+
+        SimpleNode left = null;
+        for (int i = 0; i < nodes.size(); i++) {
+            if (left == null) {
+                left = i > 0 ? nodes.get(i - 1) : null;
+            }
+            SimpleNode token = nodes.get(i);
+            SimpleNode right = i < nodes.size() - 1 ? nodes.get(i + 1) : null;
+
+            if (token instanceof LogicalExpression) {
+                LogicalExpression logical = (LogicalExpression) token;
+
+                // remember the logical operator
+                String operator = logical.getOperator().toString();
+
+                if (left == null) {
+                    throw new SimpleParserException("Logical operator " + operator + " has no left hand side token", token.getToken().getIndex());
+                }
+                if (!logical.acceptLeftNode(left)) {
+                    throw new SimpleParserException("Logical operator " + operator + " does not support left hand side token " + left.getToken(), token.getToken().getIndex());
+                }
+                if (right == null) {
+                    throw new SimpleParserException("Logical operator " + operator + " has no right hand side token", token.getToken().getIndex());
+                }
+                if (!logical.acceptRightNode(right)) {
+                    throw new SimpleParserException("Logical operator " + operator + " does not support right hand side token " + left.getToken(), token.getToken().getIndex());
+                }
+
+                // pop previous as we need to replace it with this binary operator
+                stack.pop();
+                stack.push(token);
+                // advantage after the right hand side
+                i++;
+                // this token is now the left for the next loop
+                left = token;
+            } else {
+                // clear left
+                left = null;
+                stack.push(token);
+            }
+        }
+
+        nodes.clear();
+        nodes.addAll(stack);
+    }
+
+    /**
+     * Creates the {@link Predicate}s from the AST nodes.
+     *
+     * @return the created {@link Predicate}s, is never <tt>null</tt>.
+     */
+    private List<Predicate> createPredicates() {
+        List<Predicate> answer = new ArrayList<Predicate>();
+        for (SimpleNode node : nodes) {
+            Expression exp = node.createExpression(expression);
+            if (exp != null) {
+                Predicate predicate = PredicateBuilder.toPredicate(exp);
+                answer.add(predicate);
+            }
+        }
+        return answer;
+    }
+
+    // --------------------------------------------------------------
+    // grammar
+    // --------------------------------------------------------------
+
+    // the predicate parser understands a lot more than the expression parser
+    // - single quoted = block of nodes enclosed by single quotes
+    // - double quoted = block of nodes enclosed by double quotes
+    // - single quoted with functions = block of nodes enclosed by single quotes allowing embedded functions
+    // - double quoted with functions = block of nodes enclosed by double quotes allowing embedded functions
+    // - function = simple functions such as ${body} etc
+    // - numeric = numeric value
+    // - boolean = boolean value
+    // - null = null value
+    // - unary operator = operator attached to the left hand side node
+    // - binary operator = operator attached to both the left and right hand side nodes
+    // - logical operator = operator attached to both the left and right hand side nodes
+
+    protected boolean singleQuotedLiteralWithFunctionsText() {
+        if (accept(TokenType.singleQuote)) {
+            nextToken(TokenType.singleQuote, TokenType.eol, TokenType.functionStart, TokenType.functionEnd, TokenType.escapedValue);
+            while (!token.getType().isSingleQuote() && !token.getType().isEol()) {
+                // we need to loop until we find the ending single quote, or the eol
+                nextToken(TokenType.singleQuote, TokenType.eol, TokenType.functionStart, TokenType.functionEnd, TokenType.escapedValue);
+            }
+            expect(TokenType.singleQuote);
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean singleQuotedLiteralText() {
+        if (accept(TokenType.singleQuote)) {
+            nextToken(TokenType.singleQuote, TokenType.eol);
+            while (!token.getType().isSingleQuote() && !token.getType().isEol()) {
+                // we need to loop until we find the ending single quote, or the eol
+                nextToken(TokenType.singleQuote, TokenType.eol);
+            }
+            expect(TokenType.singleQuote);
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean doubleQuotedLiteralWithFunctionsText() {
+        if (accept(TokenType.doubleQuote)) {
+            nextToken(TokenType.doubleQuote, TokenType.eol, TokenType.functionStart, TokenType.functionEnd, TokenType.escapedValue);
+            while (!token.getType().isDoubleQuote() && !token.getType().isEol()) {
+                // we need to loop until we find the ending double quote, or the eol
+                nextToken(TokenType.doubleQuote, TokenType.eol, TokenType.functionStart, TokenType.functionEnd, TokenType.escapedValue);
+            }
+            expect(TokenType.doubleQuote);
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean doubleQuotedLiteralText() {
+        if (accept(TokenType.doubleQuote)) {
+            nextToken(TokenType.doubleQuote, TokenType.eol);
+            while (!token.getType().isDoubleQuote() && !token.getType().isEol()) {
+                // we need to loop until we find the ending double quote, or the eol
+                nextToken(TokenType.doubleQuote, TokenType.eol);
+            }
+            expect(TokenType.doubleQuote);
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean functionText() {
+        if (accept(TokenType.functionStart)) {
+            nextToken(TokenType.functionEnd, TokenType.eol);
+            nextToken();
+            while (!token.getType().isFunctionEnd() && !token.getType().isEol()) {
+                // we need to loop until we find the ending function quote, or the eol
+                nextToken(TokenType.functionEnd, TokenType.eol);
+            }
+            expect(TokenType.functionEnd);
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean unaryOperator() {
+        if (accept(TokenType.unaryOperator)) {
+            nextToken();
+            // there should be a whitespace after the operator
+            expect(TokenType.whiteSpace);
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean binaryOperator() {
+        if (accept(TokenType.binaryOperator)) {
+            // remember the binary operator
+            BinaryOperatorType operatorType = BinaryOperatorType.asOperator(token.getText());
+
+            nextToken();
+            // there should be at least one whitespace after the operator
+            expectAndAcceptMore(TokenType.whiteSpace);
+
+            // okay a binary operator may not support all kind if preceding parameters, so we need to limit this
+            BinaryOperatorType.ParameterType[] types = BinaryOperatorType.supportedParameterTypes(operatorType);
+
+            // based on the parameter types the binary operator support, we need to set this state into
+            // the following booleans so we know how to proceed in the grammar
+            boolean literalWithFunctionsSupported = false;
+            boolean literalSupported = false;
+            boolean functionSupported = false;
+            boolean numericSupported = false;
+            boolean booleanSupported = false;
+            boolean nullSupported = false;
+            if (types == null || types.length == 0) {
+                literalWithFunctionsSupported = true;
+                // favor literal with functions over literals without functions
+                literalSupported = false;
+                functionSupported = true;
+                numericSupported = true;
+                booleanSupported = true;
+                nullSupported = true;
+            } else {
+                for (BinaryOperatorType.ParameterType parameterType : types) {
+                    literalSupported |= parameterType.isLiteralSupported();
+                    literalWithFunctionsSupported |= parameterType.isLiteralWithFunctionSupport();
+                    functionSupported |= parameterType.isFunctionSupport();
+                    nullSupported |= parameterType.isNumericValueSupported();
+                    booleanSupported |= parameterType.isBooleanValueSupported();
+                    nullSupported |= parameterType.isNullValueSupported();
+                }
+            }
+
+            // then we proceed in the grammar according to the parameter types supported by the given binary operator
+            //CHECKSTYLE:OFF
+            if ((literalWithFunctionsSupported && singleQuotedLiteralWithFunctionsText())
+                    || (literalWithFunctionsSupported && doubleQuotedLiteralWithFunctionsText())
+                    || (literalSupported && singleQuotedLiteralText())
+                    || (literalSupported && doubleQuotedLiteralText())
+                    || (functionSupported && functionText())
+                    || (numericSupported && numericValue())
+                    || (booleanSupported && booleanValue())
+                    || (nullSupported && nullValue())) {
+                // then after the right hand side value, there should be a whitespace if there is more tokens
+                nextToken();
+                if (!token.getType().isEol()) {
+                    expect(TokenType.whiteSpace);
+                }
+            } else {
+                throw new SimpleParserException("Binary operator " + operatorType + " does not support token " + token, token.getIndex());
+            }
+            //CHECKSTYLE:ON
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean logicalOperator() {
+        if (accept(TokenType.logicalOperator)) {
+            // remember the logical operator
+            LogicalOperatorType operatorType = LogicalOperatorType.asOperator(token.getText());
+
+            nextToken();
+            // there should be at least one whitespace after the operator
+            expectAndAcceptMore(TokenType.whiteSpace);
+
+            // then we expect either some quoted text, another function, or a numeric, boolean or null value
+            if (singleQuotedLiteralWithFunctionsText()
+                    || doubleQuotedLiteralWithFunctionsText()
+                    || functionText()
+                    || numericValue()
+                    || booleanValue()
+                    || nullValue()) {
+                // then after the right hand side value, there should be a whitespace if there is more tokens
+                nextToken();
+                if (!token.getType().isEol()) {
+                    expect(TokenType.whiteSpace);
+                }
+            } else {
+                throw new SimpleParserException("Logical operator " + operatorType + " does not support token " + token, token.getIndex());
+            }
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean numericValue() {
+        if (accept(TokenType.numericValue)) {
+            // no other tokens to check so do not use nextToken
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean booleanValue() {
+        if (accept(TokenType.booleanValue)) {
+            // no other tokens to check so do not use nextToken
+            return true;
+        }
+        return false;
+    }
+
+    protected boolean nullValue() {
+        if (accept(TokenType.nullValue)) {
+            // no other tokens to check so do not use nextToken
+            return true;
+        }
+        return false;
+    }
+
+}

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleToken.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleToken.java?rev=1167462&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleToken.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/language/simple/SimpleToken.java Sat Sep 10 05:43:14 2011
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.language.simple;
+
+/**
+ * Holder for a token, with associated type and position in the input.
+ */
+public final class SimpleToken {
+
+    private final SimpleTokenType type;
+    private final int index;
+    private final int length;
+
+    public SimpleToken(SimpleTokenType type, int index) {
+        this(type, index, type.getValue() != null ? type.getValue().length() : 0);
+    }
+
+    public SimpleToken(SimpleTokenType type, int index, int length) {
+        this.type = type;
+        this.index = index;
+        this.length = length;
+    }
+
+    public SimpleTokenType getType() {
+        return type;
+    }
+
+    public int getIndex() {
+        return index;
+    }
+
+    public String getText() {
+        return type.getValue();
+    }
+
+    public int getLength() {
+        return length;
+    }
+
+    @Override
+    public String toString() {
+        return type.toString();
+    }
+}