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 2016/12/05 11:29:51 UTC

[08/11] camel git commit: CAMEL-10559: route parser for java and xml to parse source code. Donated from fabric8 project.

CAMEL-10559: route parser for java and xml to parse source code. Donated from fabric8 project.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/adc20f70
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/adc20f70
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/adc20f70

Branch: refs/heads/parser
Commit: adc20f7083614478c10b64f2083c4600aee974f9
Parents: 9ce3719
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 5 12:17:06 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 5 12:17:06 2016 +0100

----------------------------------------------------------------------
 .../org/apache/camel/parser/CamelXmlHelper.java | 260 +++++++++++++++++++
 .../apache/camel/parser/RouteBuilderParser.java |   1 -
 .../camel/parser/XmlLineNumberParser.java       | 194 ++++++++++++++
 .../org/apache/camel/parser/XmlRouteParser.java | 141 ++++++++++
 .../parser/xml/FindElementInRoutesTest.java     |  44 ++++
 .../parser/xml/XmlOnExceptionRouteTest.java     |  55 ++++
 .../apache/camel/parser/xml/XmlRouteTest.java   |  51 ++++
 .../camel/parser/xml/mycamel-onexception.xml    |  33 +++
 .../org/apache/camel/parser/xml/mycamel.xml     |  26 ++
 .../org/apache/camel/parser/xml/myroutes.xml    |  19 ++
 10 files changed, 823 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/main/java/org/apache/camel/parser/CamelXmlHelper.java
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/main/java/org/apache/camel/parser/CamelXmlHelper.java b/tooling/route-parser/src/main/java/org/apache/camel/parser/CamelXmlHelper.java
new file mode 100644
index 0000000..b4a7848
--- /dev/null
+++ b/tooling/route-parser/src/main/java/org/apache/camel/parser/CamelXmlHelper.java
@@ -0,0 +1,260 @@
+/**
+ * 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.parser;
+
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.jboss.forge.roaster.model.util.Strings;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class CamelXmlHelper {
+
+    public static String getSafeAttribute(Node node, String key) {
+        if (node != null) {
+            Node attr = node.getAttributes().getNamedItem(key);
+            if (attr != null) {
+                return attr.getNodeValue();
+            }
+        }
+        return null;
+    }
+
+    public static List<Node> findAllEndpoints(Document dom) {
+        List<Node> nodes = new ArrayList<>();
+
+        NodeList list = dom.getElementsByTagName("endpoint");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            if ("endpoint".equals(child.getNodeName())) {
+                // it may not be a camel namespace, so skip those
+                String ns = child.getNamespaceURI();
+                if (ns == null) {
+                    NamedNodeMap attrs = child.getAttributes();
+                    if (attrs != null) {
+                        Node node = attrs.getNamedItem("xmlns");
+                        if (node != null) {
+                            ns = node.getNodeValue();
+                        }
+                    }
+                }
+                // assume no namespace its for camel
+                if (ns == null || ns.contains("camel")) {
+                    nodes.add(child);
+                }
+            }
+        }
+
+        list = dom.getElementsByTagName("onException");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            findAllUrisRecursive(child, nodes);
+        }
+        list = dom.getElementsByTagName("onCompletion");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            findAllUrisRecursive(child, nodes);
+        }
+        list = dom.getElementsByTagName("intercept");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            findAllUrisRecursive(child, nodes);
+        }
+        list = dom.getElementsByTagName("interceptFrom");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            findAllUrisRecursive(child, nodes);
+        }
+        list = dom.getElementsByTagName("interceptSendToEndpoint");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            findAllUrisRecursive(child, nodes);
+        }
+        list = dom.getElementsByTagName("rest");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            if ("route".equals(child.getNodeName()) || "to".equals(child.getNodeName())) {
+                findAllUrisRecursive(child, nodes);
+            }
+        }
+        list = dom.getElementsByTagName("route");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            if ("route".equals(child.getNodeName())) {
+                findAllUrisRecursive(child, nodes);
+            }
+        }
+
+        return nodes;
+    }
+
+    private static void findAllUrisRecursive(Node node, List<Node> nodes) {
+        // okay its a route so grab all uri attributes we can find
+        String url = getSafeAttribute(node, "uri");
+        if (url != null) {
+            nodes.add(node);
+        }
+
+        NodeList children = node.getChildNodes();
+        if (children != null) {
+            for (int i = 0; i < children.getLength(); i++) {
+                Node child = children.item(i);
+                if (child.getNodeType() == Node.ELEMENT_NODE) {
+                    findAllUrisRecursive(child, nodes);
+                }
+            }
+        }
+    }
+
+    public static List<Node> findAllSimpleExpressions(Document dom) {
+        List<Node> nodes = new ArrayList<>();
+
+        NodeList list = dom.getElementsByTagName("route");
+        for (int i = 0; i < list.getLength(); i++) {
+            Node child = list.item(i);
+            if ("route".equals(child.getNodeName())) {
+                findAllSimpleExpressionsRecursive(child, nodes);
+            }
+        }
+
+        return nodes;
+    }
+
+    private static void findAllSimpleExpressionsRecursive(Node node, List<Node> nodes) {
+        // okay its a route so grab if its <simple>
+        if ("simple".equals(node.getNodeName())) {
+            nodes.add(node);
+        }
+
+        NodeList children = node.getChildNodes();
+        if (children != null) {
+            for (int i = 0; i < children.getLength(); i++) {
+                Node child = children.item(i);
+                if (child.getNodeType() == Node.ELEMENT_NODE) {
+                    findAllSimpleExpressionsRecursive(child, nodes);
+                }
+            }
+        }
+    }
+
+    public static Element getSelectedCamelElementNode(String key, InputStream resourceInputStream) throws Exception {
+        Document root = loadCamelXmlFileAsDom(resourceInputStream);
+        Element selectedElement = null;
+        if (root != null) {
+            Node selectedNode = findCamelNodeInDocument(root, key);
+            if (selectedNode instanceof Element) {
+                selectedElement = (Element) selectedNode;
+            }
+        }
+        return selectedElement;
+    }
+
+    private static Document loadCamelXmlFileAsDom(InputStream resourceInputStream) throws Exception {
+        // TODO:
+        Document root = XmlLineNumberParser.parseXml(resourceInputStream, "camelContext,routes,rests", "http://camel.apache.org/schema/spring");
+        return root;
+    }
+
+    private static Node findCamelNodeInDocument(Document root, String key) {
+        Node selectedNode = null;
+        if (root != null && !Strings.isBlank(key)) {
+            String[] paths = key.split("/");
+            NodeList camels = getCamelContextElements(root);
+            if (camels != null) {
+                Map<String, Integer> rootNodeCounts = new HashMap<>();
+                for (int i = 0, size = camels.getLength(); i < size; i++) {
+                    Node node = camels.item(i);
+                    boolean first = true;
+                    for (String path : paths) {
+                        if (first) {
+                            first = false;
+                            String actual = getIdOrIndex(node, rootNodeCounts);
+                            if (!equal(actual, path)) {
+                                node = null;
+                            }
+                        } else {
+                            node = findCamelNodeForPath(node, path);
+                        }
+                        if (node == null) {
+                            break;
+                        }
+                    }
+                    if (node != null) {
+                        return node;
+                    }
+                }
+            }
+        }
+        return selectedNode;
+    }
+
+    private static Node findCamelNodeForPath(Node node, String path) {
+        NodeList childNodes = node.getChildNodes();
+        if (childNodes != null) {
+            Map<String, Integer> nodeCounts = new HashMap<>();
+            for (int i = 0, size = childNodes.getLength(); i < size; i++) {
+                Node child = childNodes.item(i);
+                if (child instanceof Element) {
+                    String actual = getIdOrIndex(child, nodeCounts);
+                    if (equal(actual, path)) {
+                        return child;
+                    }
+                }
+            }
+        }
+        return null;
+    }
+
+    private static String getIdOrIndex(Node node, Map<String, Integer> nodeCounts) {
+        String answer = null;
+        if (node instanceof Element) {
+            Element element = (Element) node;
+            String elementName = element.getTagName();
+            if ("routes".equals(elementName)) {
+                elementName = "camelContext";
+            }
+            Integer countObject = nodeCounts.get(elementName);
+            int count = countObject != null ? countObject.intValue() : 0;
+            nodeCounts.put(elementName, ++count);
+            answer = element.getAttribute("id");
+            if (Strings.isBlank(answer)) {
+                answer = "_" + elementName + count;
+            }
+        }
+        return answer;
+    }
+
+    private static NodeList getCamelContextElements(Document dom) {
+        NodeList camels = dom.getElementsByTagName("camelContext");
+        if (camels == null || camels.getLength() == 0) {
+            camels = dom.getElementsByTagName("routes");
+        }
+        return camels;
+    }
+
+    private static boolean equal(Object a, Object b) {
+        return a == b?true:a != null && b != null && a.equals(b);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/main/java/org/apache/camel/parser/RouteBuilderParser.java
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/main/java/org/apache/camel/parser/RouteBuilderParser.java b/tooling/route-parser/src/main/java/org/apache/camel/parser/RouteBuilderParser.java
index 2327ae1..b84888a 100644
--- a/tooling/route-parser/src/main/java/org/apache/camel/parser/RouteBuilderParser.java
+++ b/tooling/route-parser/src/main/java/org/apache/camel/parser/RouteBuilderParser.java
@@ -277,5 +277,4 @@ public class RouteBuilderParser {
         return null;
     }
 
-
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlLineNumberParser.java
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlLineNumberParser.java b/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlLineNumberParser.java
new file mode 100644
index 0000000..c2fffce
--- /dev/null
+++ b/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlLineNumberParser.java
@@ -0,0 +1,194 @@
+/**
+ * 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.parser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.util.Stack;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.xml.sax.Attributes;
+import org.xml.sax.InputSource;
+import org.xml.sax.Locator;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+/**
+ * An XML parser that uses SAX to include line and column number for each XML element in the parsed Document.
+ * <p/>
+ * The line number and column number can be obtained from a Node/Element using
+ * <pre>
+ *   String lineNumber = (String) node.getUserData(XmlLineNumberParser.LINE_NUMBER);
+ *   String lineNumberEnd = (String) node.getUserData(XmlLineNumberParser.LINE_NUMBER_END);
+ *   String columnNumber = (String) node.getUserData(XmlLineNumberParser.COLUMN_NUMBER);
+ *   String columnNumberEnd = (String) node.getUserData(XmlLineNumberParser.COLUMN_NUMBER_END);
+ * </pre>
+ */
+public final class XmlLineNumberParser {
+
+    public static final String LINE_NUMBER = "lineNumber";
+    public static final String COLUMN_NUMBER = "colNumber";
+    public static final String LINE_NUMBER_END = "lineNumberEnd";
+    public static final String COLUMN_NUMBER_END = "colNumberEnd";
+
+    /**
+     * Parses the XML.
+     *
+     * @param is the XML content as an input stream
+     * @return the DOM model
+     * @throws Exception is thrown if error parsing
+     */
+    public static Document parseXml(final InputStream is) throws Exception {
+        return parseXml(is, null, null);
+    }
+
+    /**
+     * Parses the XML.
+     *
+     * @param is the XML content as an input stream
+     * @param rootNames one or more root names that is used as baseline for beginning the parsing, for example camelContext to start parsing
+     *                  when Camel is discovered. Multiple names can be defined separated by comma
+     * @param forceNamespace an optional namespace to force assign to each node. This may be needed for JAXB unmarshalling from XML -> POJO.
+     * @return the DOM model
+     * @throws Exception is thrown if error parsing
+     */
+    public static Document parseXml(final InputStream is, final String rootNames, final String forceNamespace) throws Exception {
+        final Document doc;
+        SAXParser parser;
+        final SAXParserFactory factory = SAXParserFactory.newInstance();
+        parser = factory.newSAXParser();
+        final DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+        // turn off validator and loading external dtd
+        dbf.setValidating(false);
+        dbf.setNamespaceAware(true);
+        dbf.setFeature("http://xml.org/sax/features/namespaces", false);
+        dbf.setFeature("http://xml.org/sax/features/validation", false);
+        dbf.setFeature("http://apache.org/xml/features/nonvalidating/load-dtd-grammar", false);
+        dbf.setFeature("http://apache.org/xml/features/nonvalidating/load-external-dtd", false);
+        dbf.setFeature("http://xml.org/sax/features/external-parameter-entities", false);
+        dbf.setFeature("http://xml.org/sax/features/external-general-entities", false);
+        final DocumentBuilder docBuilder = dbf.newDocumentBuilder();
+        doc = docBuilder.newDocument();
+
+        final Stack<Element> elementStack = new Stack<Element>();
+        final StringBuilder textBuffer = new StringBuilder();
+        final DefaultHandler handler = new DefaultHandler() {
+            private Locator locator;
+            private boolean found;
+
+            @Override
+            public void setDocumentLocator(final Locator locator) {
+                this.locator = locator; // Save the locator, so that it can be used later for line tracking when traversing nodes.
+                this.found = rootNames == null;
+            }
+
+            private boolean isRootName(String qName) {
+                for (String root : rootNames.split(",")) {
+                    if (qName.equals(root)) {
+                        return true;
+                    }
+                }
+                return false;
+            }
+
+            @Override
+            public void startElement(final String uri, final String localName, final String qName, final Attributes attributes) throws SAXException {
+                addTextIfNeeded();
+
+                if (rootNames != null && !found) {
+                    if (isRootName(qName)) {
+                        found = true;
+                    }
+                }
+
+                if (found) {
+                    Element el;
+                    if (forceNamespace != null) {
+                        el = doc.createElementNS(forceNamespace, qName);
+                    } else {
+                        el = doc.createElement(qName);
+                    }
+
+                    for (int i = 0; i < attributes.getLength(); i++) {
+                        el.setAttribute(attributes.getQName(i), attributes.getValue(i));
+                    }
+
+                    el.setUserData(LINE_NUMBER, String.valueOf(this.locator.getLineNumber()), null);
+                    el.setUserData(COLUMN_NUMBER, String.valueOf(this.locator.getColumnNumber()), null);
+                    elementStack.push(el);
+                }
+            }
+
+            @Override
+            public void endElement(final String uri, final String localName, final String qName) {
+                if (!found) {
+                    return;
+                }
+
+                addTextIfNeeded();
+
+                final Element closedEl = elementStack.isEmpty() ? null : elementStack.pop();
+                if (closedEl != null) {
+                    if (elementStack.isEmpty()) {
+                        // Is this the root element?
+                        doc.appendChild(closedEl);
+                    } else {
+                        final Element parentEl = elementStack.peek();
+                        parentEl.appendChild(closedEl);
+                    }
+
+                    closedEl.setUserData(LINE_NUMBER_END, String.valueOf(this.locator.getLineNumber()), null);
+                    closedEl.setUserData(COLUMN_NUMBER_END, String.valueOf(this.locator.getColumnNumber()), null);
+                }
+            }
+
+            @Override
+            public void characters(final char ch[], final int start, final int length) throws SAXException {
+                textBuffer.append(ch, start, length);
+            }
+
+            @Override
+            public InputSource resolveEntity(String publicId, String systemId) throws IOException, SAXException {
+                // do not resolve external dtd
+                return new InputSource(new StringReader(""));
+            }
+
+            // Outputs text accumulated under the current node
+            private void addTextIfNeeded() {
+                if (textBuffer.length() > 0) {
+                    final Element el = elementStack.isEmpty() ? null : elementStack.peek();
+                    if (el != null) {
+                        final Node textNode = doc.createTextNode(textBuffer.toString());
+                        el.appendChild(textNode);
+                        textBuffer.delete(0, textBuffer.length());
+                    }
+                }
+            }
+        };
+        parser.parse(is, handler);
+
+        return doc;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlRouteParser.java
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlRouteParser.java b/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlRouteParser.java
new file mode 100644
index 0000000..1b206cb
--- /dev/null
+++ b/tooling/route-parser/src/main/java/org/apache/camel/parser/XmlRouteParser.java
@@ -0,0 +1,141 @@
+/**
+ * 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.parser;
+
+import java.io.InputStream;
+import java.util.List;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Node;
+
+import org.apache.camel.parser.model.CamelEndpointDetails;
+import org.apache.camel.parser.model.CamelSimpleDetails;
+import org.jboss.forge.roaster.model.util.Strings;
+
+import static org.apache.camel.parser.CamelXmlHelper.getSafeAttribute;
+
+public class XmlRouteParser {
+
+    public static void parseXmlRouteEndpoints(InputStream xml, String baseDir, String fullyQualifiedFileName,
+                                              List<CamelEndpointDetails> endpoints) throws Exception {
+
+        // find all the endpoints (currently only <endpoint> and within <route>)
+        // try parse it as dom
+        Document dom = null;
+        try {
+            dom = XmlLineNumberParser.parseXml(xml);
+        } catch (Exception e) {
+            // ignore as the xml file may not be valid at this point
+        }
+        if (dom != null) {
+            List<Node> nodes = CamelXmlHelper.findAllEndpoints(dom);
+            for (Node node : nodes) {
+                String uri = getSafeAttribute(node, "uri");
+                if (uri != null) {
+                    // trim and remove whitespace noise
+                    uri = trimEndpointUri(uri);
+                }
+                if (!Strings.isBlank(uri)) {
+                    String id = getSafeAttribute(node, "id");
+                    String lineNumber = (String) node.getUserData(XmlLineNumberParser.LINE_NUMBER);
+                    String lineNumberEnd = (String) node.getUserData(XmlLineNumberParser.LINE_NUMBER_END);
+
+                    // we only want the relative dir name from the resource directory, eg META-INF/spring/foo.xml
+                    String fileName = fullyQualifiedFileName;
+                    if (fileName.startsWith(baseDir)) {
+                        fileName = fileName.substring(baseDir.length() + 1);
+                    }
+
+                    boolean consumerOnly = false;
+                    boolean producerOnly = false;
+                    String nodeName = node.getNodeName();
+                    if ("from".equals(nodeName) || "pollEnrich".equals(nodeName)) {
+                        consumerOnly = true;
+                    } else if ("to".equals(nodeName) || "enrich".equals(nodeName) || "wireTap".equals(nodeName)) {
+                        producerOnly = true;
+                    }
+
+                    CamelEndpointDetails detail = new CamelEndpointDetails();
+                    detail.setFileName(fileName);
+                    detail.setLineNumber(lineNumber);
+                    detail.setLineNumberEnd(lineNumberEnd);
+                    detail.setEndpointInstance(id);
+                    detail.setEndpointUri(uri);
+                    detail.setEndpointComponentName(endpointComponentName(uri));
+                    detail.setConsumerOnly(consumerOnly);
+                    detail.setProducerOnly(producerOnly);
+                    endpoints.add(detail);
+                }
+            }
+        }
+    }
+
+    public static void parseXmlRouteSimpleExpressions(InputStream xml, String baseDir, String fullyQualifiedFileName,
+                                                      List<CamelSimpleDetails> simpleExpressions) throws Exception {
+
+        // find all the simple expressions
+        // try parse it as dom
+        Document dom = null;
+        try {
+            dom = XmlLineNumberParser.parseXml(xml);
+        } catch (Exception e) {
+            // ignore as the xml file may not be valid at this point
+        }
+        if (dom != null) {
+            List<Node> nodes = CamelXmlHelper.findAllSimpleExpressions(dom);
+            for (Node node : nodes) {
+                String simple = node.getTextContent();
+                String lineNumber = (String) node.getUserData(XmlLineNumberParser.LINE_NUMBER);
+                String lineNumberEnd = (String) node.getUserData(XmlLineNumberParser.LINE_NUMBER_END);
+
+                // we only want the relative dir name from the resource directory, eg META-INF/spring/foo.xml
+                String fileName = fullyQualifiedFileName;
+                if (fileName.startsWith(baseDir)) {
+                    fileName = fileName.substring(baseDir.length() + 1);
+                }
+
+                CamelSimpleDetails detail = new CamelSimpleDetails();
+                detail.setFileName(fileName);
+                detail.setLineNumber(lineNumber);
+                detail.setLineNumberEnd(lineNumberEnd);
+                detail.setSimple(simple);
+                simpleExpressions.add(detail);
+            }
+        }
+    }
+
+    private static String endpointComponentName(String uri) {
+        if (uri != null) {
+            int idx = uri.indexOf(":");
+            if (idx > 0) {
+                return uri.substring(0, idx);
+            }
+        }
+        return null;
+    }
+
+
+    private static String trimEndpointUri(String uri) {
+        uri = uri.trim();
+        // if the uri is using new-lines then remove whitespace noise before & and ? separator
+        uri = uri.replaceAll("(\\s+)(\\&)", "$2");
+        uri = uri.replaceAll("(\\&)(\\s+)", "$1");
+        uri = uri.replaceAll("(\\?)(\\s+)", "$1");
+        return uri;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/FindElementInRoutesTest.java
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/FindElementInRoutesTest.java b/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/FindElementInRoutesTest.java
new file mode 100644
index 0000000..b39fc2d
--- /dev/null
+++ b/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/FindElementInRoutesTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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.parser.xml;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+
+import org.apache.camel.parser.CamelXmlHelper;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Element;
+
+import static org.junit.Assert.assertNotNull;
+
+public class FindElementInRoutesTest {
+
+    private static final Logger LOG = LoggerFactory.getLogger(FindElementInRoutesTest.class);
+
+    @Test
+    public void testXml() throws Exception {
+        InputStream is = new FileInputStream("src/test/resources/org/apache/camel/parser/xml/myroutes.xml");
+        String key = "_camelContext1/cbr-route/_from1";
+        Element element = CamelXmlHelper.getSelectedCamelElementNode(key, is);
+        assertNotNull("Could not find Element for key " + key, element);
+
+        LOG.info("Found element " + element.getTagName());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlOnExceptionRouteTest.java
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlOnExceptionRouteTest.java b/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlOnExceptionRouteTest.java
new file mode 100644
index 0000000..6f8aeb6
--- /dev/null
+++ b/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlOnExceptionRouteTest.java
@@ -0,0 +1,55 @@
+/**
+ * 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.parser.xml;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.camel.parser.XmlRouteParser;
+import org.apache.camel.parser.model.CamelEndpointDetails;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class XmlOnExceptionRouteTest {
+
+    private static final Logger LOG = LoggerFactory.getLogger(XmlOnExceptionRouteTest.class);
+
+    @Test
+    public void testXml() throws Exception {
+        List<CamelEndpointDetails> endpoints = new ArrayList<>();
+
+        InputStream is = new FileInputStream("src/test/resources/org/apache/camel/parser/xml/mycamel-onexception.xml");
+        String fqn = "src/test/resources/org/apache/camel/parser/xml/mycamel-onexception.xml";
+        String baseDir = "src/test/resources";
+        XmlRouteParser.parseXmlRouteEndpoints(is, baseDir, fqn, endpoints);
+
+        for (CamelEndpointDetails detail : endpoints) {
+            LOG.info(detail.getEndpointUri());
+        }
+
+        Assert.assertEquals("log:all", endpoints.get(0).getEndpointUri());
+        Assert.assertEquals("mock:dead", endpoints.get(1).getEndpointUri());
+        Assert.assertEquals("log:done", endpoints.get(2).getEndpointUri());
+        Assert.assertEquals("stream:in?promptMessage=Enter something:", endpoints.get(3).getEndpointUri());
+        Assert.assertEquals("stream:out", endpoints.get(4).getEndpointUri());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlRouteTest.java
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlRouteTest.java b/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlRouteTest.java
new file mode 100644
index 0000000..05be852
--- /dev/null
+++ b/tooling/route-parser/src/test/java/org/apache/camel/parser/xml/XmlRouteTest.java
@@ -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.parser.xml;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.camel.parser.XmlRouteParser;
+import org.apache.camel.parser.model.CamelEndpointDetails;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class XmlRouteTest {
+
+    private static final Logger LOG = LoggerFactory.getLogger(XmlRouteTest.class);
+
+    @Test
+    public void testXml() throws Exception {
+        List<CamelEndpointDetails> endpoints = new ArrayList<>();
+
+        InputStream is = new FileInputStream("src/test/resources/org/apache/camel/parser/xml/mycamel.xml");
+        String fqn = "src/test/resources/org/apache/camel/camel/parser/xml/mycamel.xml";
+        String baseDir = "src/test/resources";
+        XmlRouteParser.parseXmlRouteEndpoints(is, baseDir, fqn, endpoints);
+
+        for (CamelEndpointDetails detail : endpoints) {
+            LOG.info(detail.getEndpointUri());
+        }
+        Assert.assertEquals("stream:in?promptMessage=Enter something:", endpoints.get(0).getEndpointUri());
+        Assert.assertEquals("stream:out", endpoints.get(1).getEndpointUri());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel-onexception.xml
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel-onexception.xml b/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel-onexception.xml
new file mode 100644
index 0000000..14a370f
--- /dev/null
+++ b/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel-onexception.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd">
+
+  <camelContext xmlns="http://camel.apache.org/schema/spring">
+
+    <endpoint id="logger" uri="log:all"/>
+
+    <onException>
+      <exception>java.lang.Exception</exception>
+      <handled>
+        <constant>true</constant>
+      </handled>
+      <to uri="mock:dead"/>
+    </onException>
+
+    <onCompletion>
+      <to uri="log:done"/>
+    </onCompletion>
+
+    <route>
+      <from uri="stream:in?promptMessage=Enter something: "/>
+      <transform>
+        <simple>Hello ${body.toUpperCase()}</simple>
+      </transform>
+      <to uri="stream:out"/>
+    </route>
+  </camelContext>
+
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel.xml
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel.xml b/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel.xml
new file mode 100644
index 0000000..f243369
--- /dev/null
+++ b/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/mycamel.xml
@@ -0,0 +1,26 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd">
+
+  <!-- START SNIPPET: e1 -->
+  <!-- camelContext is the Camel runtime, where we can host Camel routes -->
+  <camelContext xmlns="http://camel.apache.org/schema/spring">
+    <route>
+      <!-- read input from the console using the stream component -->
+      <from
+          uri="stream:in?promptMessage=Enter something: "/>
+      <!-- transform the input to upper case using the simple language -->
+      <!-- you can also use other languages such as groovy, ognl, mvel, javascript etc. -->
+      <transform>
+        <simple>Hello ${body.toUpperCase()}</simple>
+      </transform>
+      <!-- and then print to the console -->
+      <to uri="stream:out"/>
+    </route>
+  </camelContext>
+  <!-- END SNIPPET: e1 -->
+
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/adc20f70/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/myroutes.xml
----------------------------------------------------------------------
diff --git a/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/myroutes.xml b/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/myroutes.xml
new file mode 100644
index 0000000..c132e34
--- /dev/null
+++ b/tooling/route-parser/src/test/resources/org/apache/camel/parser/xml/myroutes.xml
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<routes xmlns="http://camel.apache.org/schema/spring">
+    <route id="cbr-route">
+      <from uri="timer:foo?period=5000"/>
+      <!-- generate random number message, using a 3 digit number -->
+      <transform>
+        <method bean="myTransformer"/>
+      </transform>
+      <choice>
+        <when>
+          <simple>${body} &gt; 500</simple>
+          <log message="High priority message : ${body}"/>
+        </when>
+        <otherwise>
+          <log message="Low priority message  : ${body}"/>
+        </otherwise>
+      </choice>
+    </route>
+</routes>