You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by gg...@apache.org on 2023/05/19 11:04:57 UTC

[camel] 01/06: [CAMEL-18189] Add XmlStreamDetector to prevent extra XML parsing

This is an automated email from the ASF dual-hosted git repository.

ggrzybek pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/camel.git

commit bb8aa23f4c3a8a16e3335c562bce015ba43bb0ef
Author: Grzegorz Grzybek <gr...@gmail.com>
AuthorDate: Tue May 2 10:34:45 2023 +0200

    [CAMEL-18189] Add XmlStreamDetector to prevent extra XML parsing
---
 core/camel-xml-io-util/pom.xml                     |  12 ++
 .../camel/xml/io/util/XmlStreamDetector.java       | 159 ++++++++++++++++++++
 .../apache/camel/xml/io/util/XmlStreamInfo.java    |  81 +++++++++++
 .../camel/xml/io/util/XmlStreamDetectorTest.java   | 161 +++++++++++++++++++++
 .../java/org/apache/camel/xml/in/ParserTest.java   | 147 +++++++++++++++++++
 .../apache/camel/dsl/jbang/core/commands/Run.java  |  22 ++-
 .../camel/dsl/xml/io/XmlRoutesBuilderLoader.java   |  48 ++++--
 7 files changed, 614 insertions(+), 16 deletions(-)

diff --git a/core/camel-xml-io-util/pom.xml b/core/camel-xml-io-util/pom.xml
index a59886a8a46..c55b4ea1e79 100644
--- a/core/camel-xml-io-util/pom.xml
+++ b/core/camel-xml-io-util/pom.xml
@@ -35,4 +35,16 @@
         <firstVersion>3.10.0</firstVersion>
         <label>core,xml</label>
     </properties>
+
+    <dependencies>
+
+        <!-- testing -->
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
 </project>
diff --git a/core/camel-xml-io-util/src/main/java/org/apache/camel/xml/io/util/XmlStreamDetector.java b/core/camel-xml-io-util/src/main/java/org/apache/camel/xml/io/util/XmlStreamDetector.java
new file mode 100644
index 00000000000..68775cf60eb
--- /dev/null
+++ b/core/camel-xml-io-util/src/main/java/org/apache/camel/xml/io/util/XmlStreamDetector.java
@@ -0,0 +1,159 @@
+/*
+ * 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.xml.io.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamConstants;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
+
+/**
+ * <p>
+ * A utility class to determine as quickly as possible (without reading entire stream) important information about an
+ * XML document. Most importantly we can collect:
+ * <ul>
+ * <li>name and namespace of root element</li>
+ * <li>root element attributes and values</li>
+ * <li>prefix:namespace mapping declared at root element</li>
+ * <li>modeline declarations before root element</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * While we can have any kind of XML document and the namespaced content may be available at various places in the
+ * document, most <em>sane</em> documents can be examined simply by looking at the root element. This can help e.g.,
+ * with <code>jbang run camel@camel run</code> to quickly detect what kind of XML document we're trying to <em>run</em>.
+ * This can speed later, full parsing, because we know upfront what's in the doc.
+ * </p>
+ */
+public class XmlStreamDetector {
+
+    private final XMLStreamReader reader;
+    private final XmlStreamInfo information = new XmlStreamInfo();
+
+    /**
+     * Creates a detector for XML stream. The {@link InputStream stream} should be managed (like try-resources)
+     * externally.
+     *
+     * @param  xmlStream   XML to collect information from
+     * @throws IOException thrown if there is a problem reading the file.
+     */
+    public XmlStreamDetector(final InputStream xmlStream) throws IOException {
+        if (xmlStream == null) {
+            reader = null;
+            information.problem = new IllegalArgumentException("XML Stream is null");
+            return;
+        }
+        try {
+            XMLInputFactory factory = XMLInputFactory.newInstance();
+            factory.setProperty(XMLInputFactory.IS_COALESCING, Boolean.TRUE);
+            reader = factory.createXMLStreamReader(xmlStream);
+        } catch (XMLStreamException e) {
+            information.problem = e;
+            throw new IOException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Performs the analysis of the XML Stream and returns relevant {@link XmlStreamInfo XML stream information}.
+     *
+     * @return
+     * @throws IOException
+     */
+    public XmlStreamInfo information() throws IOException {
+        if (information.problem != null) {
+            return information;
+        }
+
+        if (XMLStreamConstants.START_DOCUMENT != reader.getEventType()) {
+            information.problem = new IllegalStateException("Expected START_DOCUMENT");
+            return information;
+        }
+
+        boolean skipComments = false;
+        try {
+            while (reader.hasNext()) {
+                int ev = reader.next();
+                switch (ev) {
+                    case XMLStreamConstants.COMMENT:
+                        if (!skipComments) {
+                            // search for modelines
+                            String comment = reader.getText();
+                            if (comment != null) {
+                                comment.lines().map(String::trim).forEach(l -> {
+                                    if (l.startsWith("camel-k:")) {
+                                        information.modelines.add(l);
+                                    }
+                                });
+                            }
+                        }
+                        break;
+                    case XMLStreamConstants.START_ELEMENT:
+                        if (information.rootElementName != null) {
+                            // only root element is checked. No need to parse more
+                            return information;
+                        }
+                        skipComments = true;
+                        information.rootElementName = reader.getLocalName();
+                        information.rootElementNamespace = reader.getNamespaceURI();
+
+                        for (int ns = 0; ns < reader.getNamespaceCount(); ns++) {
+                            String prefix = reader.getNamespacePrefix(ns);
+                            information.namespaceMapping.put(prefix == null ? "" : prefix, reader.getNamespaceURI(ns));
+                        }
+                        for (int at = 0; at < reader.getAttributeCount(); at++) {
+                            QName qn = reader.getAttributeName(at);
+                            String prefix = qn.getPrefix() == null ? "" : qn.getPrefix().trim();
+                            String nsURI = qn.getNamespaceURI() == null ? "" : qn.getNamespaceURI().trim();
+                            String value = reader.getAttributeValue(at);
+                            String localPart = qn.getLocalPart();
+                            if ("".equals(nsURI) || "".equals(prefix)) {
+                                // according to XML spec, this attribut is not namespaced, not in default namespace
+                                // https://www.w3.org/TR/xml-names/#defaulting
+                                // > The namespace name for an unprefixed attribute name always has no value.
+                                information.attributes.put(localPart, value);
+                            } else {
+                                information.attributes.put("{" + nsURI + "}" + localPart, value);
+                                information.attributes.put(prefix + ":" + localPart, value);
+                            }
+                        }
+                        break;
+                    case XMLStreamConstants.END_ELEMENT:
+                    case XMLStreamConstants.END_DOCUMENT:
+                        if (information.rootElementName == null) {
+                            information.problem = new IllegalArgumentException("XML Stream is empty");
+                            return information;
+                        }
+                        break;
+                    default:
+                        break;
+                }
+            }
+        } catch (XMLStreamException e) {
+            information.problem = e;
+            return information;
+        }
+
+        return information;
+    }
+
+}
diff --git a/core/camel-xml-io-util/src/main/java/org/apache/camel/xml/io/util/XmlStreamInfo.java b/core/camel-xml-io-util/src/main/java/org/apache/camel/xml/io/util/XmlStreamInfo.java
new file mode 100644
index 00000000000..d6c8543fa44
--- /dev/null
+++ b/core/camel-xml-io-util/src/main/java/org/apache/camel/xml/io/util/XmlStreamInfo.java
@@ -0,0 +1,81 @@
+/*
+ * 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.xml.io.util;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <p>
+ * Generic information about XML Stream to make later, full parsing easier (or unnecessary if the stream is not
+ * recognized for example).
+ * </p>
+ */
+public class XmlStreamInfo {
+
+    /** Indication that there's some critical problem with the stream and it should not be handled normally */
+    Throwable problem;
+
+    String rootElementName;
+    String rootElementNamespace;
+
+    /** Prefix to namespace mapping. default prefix is available as empty String (and not as null) */
+    Map<String, String> namespaceMapping = new HashMap<>();
+
+    /**
+     * Attributes of the root element. Keys are full qualified names of the attributes and each attribute may be
+     * available as two keys: {@code prefix:localName} or {@code {namespaceURI}localName}
+     */
+    Map<String, String> attributes = new HashMap<>();
+
+    /**
+     * Trimmed and unparsed lines starting with Camel-recognized modeline markers (now: {@code camel-k:}).
+     */
+    List<String> modelines = new ArrayList<>();
+
+    public boolean isValid() {
+        return problem == null;
+    }
+
+    public Throwable getProblem() {
+        return problem;
+    }
+
+    public String getRootElementName() {
+        return rootElementName;
+    }
+
+    public String getRootElementNamespace() {
+        return rootElementNamespace;
+    }
+
+    public Map<String, String> getNamespaces() {
+        return namespaceMapping;
+    }
+
+    public Map<String, String> getAttributes() {
+        return attributes;
+    }
+
+    public List<String> getModelines() {
+        return modelines;
+    }
+
+}
diff --git a/core/camel-xml-io-util/src/test/java/org/apache/camel/xml/io/util/XmlStreamDetectorTest.java b/core/camel-xml-io-util/src/test/java/org/apache/camel/xml/io/util/XmlStreamDetectorTest.java
new file mode 100644
index 00000000000..f7deff662e3
--- /dev/null
+++ b/core/camel-xml-io-util/src/test/java/org/apache/camel/xml/io/util/XmlStreamDetectorTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.xml.io.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class XmlStreamDetectorTest {
+
+    @Test
+    public void nonExistingDocument() throws IOException {
+        XmlStreamDetector detector = new XmlStreamDetector(getClass().getResourceAsStream("non-existing"));
+        assertFalse(detector.information().isValid());
+    }
+
+    @Test
+    public void emptyDocument() throws IOException {
+        XmlStreamDetector detector = new XmlStreamDetector(new ByteArrayInputStream(new byte[0]));
+        assertFalse(detector.information().isValid());
+    }
+
+    @Test
+    public void simplestDocument() throws IOException {
+        String xml = "<root />";
+        XmlStreamDetector detector
+                = new XmlStreamDetector(new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8)));
+        XmlStreamInfo info = detector.information();
+        assertTrue(info.isValid());
+        assertEquals("root", info.getRootElementName());
+        assertNull(info.getRootElementNamespace());
+    }
+
+    @Test
+    public void documentFullOfNamespaces() throws IOException {
+        String xml = """
+                <root xmlns="urn:camel"
+                      xmlns:c="urn:camel:ns1"
+                      xmlns:d="urn:camel:ns2"
+                      xmlnS="typo"
+                      a1="v1"
+                      c:a1="v2"
+                      d:a1="v3" />
+                """;
+        XmlStreamDetector detector
+                = new XmlStreamDetector(new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8)));
+        XmlStreamInfo info = detector.information();
+        assertTrue(info.isValid());
+        assertEquals("root", info.getRootElementName());
+        assertEquals("urn:camel", info.getRootElementNamespace());
+
+        assertEquals(6, info.getAttributes().size());
+        assertEquals("typo", info.getAttributes().get("xmlnS"));
+        assertEquals("v1", info.getAttributes().get("a1"));
+        assertEquals("v2", info.getAttributes().get("c:a1"));
+        assertEquals("v2", info.getAttributes().get("{urn:camel:ns1}a1"));
+        assertEquals("v3", info.getAttributes().get("d:a1"));
+        assertEquals("v3", info.getAttributes().get("{urn:camel:ns2}a1"));
+
+        assertEquals(3, info.getNamespaces().size());
+        assertEquals("urn:camel", info.getNamespaces().get(""));
+        assertEquals("urn:camel:ns1", info.getNamespaces().get("c"));
+        assertEquals("urn:camel:ns2", info.getNamespaces().get("d"));
+    }
+
+    @Test
+    public void documentWithModeline() throws IOException {
+        String xml = """
+                <?xml version="1.0" encoding="utf-8"?>
+                <!--
+                    This is my Camel application and I'm proud of it
+                    camel-k: dependency=mvn:com.i-heart-camel:best-routes-ever:1.0.0
+                    camel-k: env=HELLO=world
+                -->
+                <!--
+                    camel-k: name=MyApplication
+                -->
+                <routes xmlns="http://camel.apache.org/schema/spring">
+                </routes>
+                """;
+        XmlStreamDetector detector
+                = new XmlStreamDetector(new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8)));
+        XmlStreamInfo info = detector.information();
+        assertTrue(info.isValid());
+        assertEquals("routes", info.getRootElementName());
+        assertEquals("http://camel.apache.org/schema/spring", info.getRootElementNamespace());
+
+        assertEquals(0, info.getAttributes().size());
+
+        assertEquals(1, info.getNamespaces().size());
+        assertEquals("http://camel.apache.org/schema/spring", info.getNamespaces().get(""));
+
+        assertEquals(3, info.getModelines().size());
+        assertEquals("camel-k: dependency=mvn:com.i-heart-camel:best-routes-ever:1.0.0", info.getModelines().get(0));
+        assertEquals("camel-k: env=HELLO=world", info.getModelines().get(1));
+        assertEquals("camel-k: name=MyApplication", info.getModelines().get(2));
+    }
+
+    @Test
+    public void simpleRoute() throws IOException {
+        String xml = """
+                <?xml version="1.0" encoding="UTF-8"?>
+                    <!-- camel-k: language=xml -->
+
+                    <routes xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+                            xmlns="http://camel.apache.org/schema/spring"
+                            xsi:schemaLocation="
+                                http://camel.apache.org/schema/spring
+                                https://camel.apache.org/schema/spring/camel-spring.xsd">
+
+                        <!-- Write your routes here, for example: -->
+                        <route id="xml1">
+                            <from uri="timer:xml1?period={{time:1000}}"/>
+                            <setBody>
+                                <simple>Hello Camel (1) from ${routeId}</simple>
+                            </setBody>
+                            <log message="${body}"/>
+                        </route>
+
+                    </routes>
+                """;
+        XmlStreamDetector detector
+                = new XmlStreamDetector(new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8)));
+        XmlStreamInfo info = detector.information();
+        assertTrue(info.isValid());
+        assertEquals("routes", info.getRootElementName());
+        assertEquals("http://camel.apache.org/schema/spring", info.getRootElementNamespace());
+
+        assertEquals(2, info.getAttributes().size());
+        assertTrue(info.getAttributes().get("xsi:schemaLocation")
+                .contains("https://camel.apache.org/schema/spring/camel-spring.xsd"));
+        assertTrue(info.getAttributes().get("{http://www.w3.org/2001/XMLSchema-instance}schemaLocation")
+                .contains("https://camel.apache.org/schema/spring/camel-spring.xsd"));
+
+        assertEquals(2, info.getNamespaces().size());
+        assertEquals("http://camel.apache.org/schema/spring", info.getNamespaces().get(""));
+        assertEquals("http://www.w3.org/2001/XMLSchema-instance", info.getNamespaces().get("xsi"));
+    }
+
+}
diff --git a/core/camel-xml-io/src/test/java/org/apache/camel/xml/in/ParserTest.java b/core/camel-xml-io/src/test/java/org/apache/camel/xml/in/ParserTest.java
new file mode 100644
index 00000000000..b5e2c44050e
--- /dev/null
+++ b/core/camel-xml-io/src/test/java/org/apache/camel/xml/in/ParserTest.java
@@ -0,0 +1,147 @@
+/*
+ * 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.xml.in;
+
+import java.io.IOException;
+import java.io.StringReader;
+
+import org.apache.camel.xml.io.MXParser;
+import org.apache.camel.xml.io.XmlPullParserException;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class ParserTest {
+
+    @Test
+    public void justParse() throws XmlPullParserException, IOException {
+        String xml = """
+                <?xml version='1.0'?>
+                <c:root xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:c="uri:camel" xmlns="uri:camel-beans">
+                    <c:e1 a="value-1" b:a="value-2" xmlns:c="uri:cxf" xmlns:b="uri:b" />
+                    <watch-out-for-entities>&nbsp;</watch-out-for-entities>
+                </c:root>
+                """;
+        BaseParser p = new BaseParser(new StringReader(xml));
+        MXParser xpp = p.parser;
+        xpp.defineEntityReplacementText("nbsp", "—");
+        int eventType = xpp.getEventType();
+        while (eventType != MXParser.END_DOCUMENT) {
+            xpp.getStartLineNumber();
+            xpp.getLineNumber();
+            xpp.getColumnNumber();
+            xpp.getDepth();
+            xpp.getPositionDescription();
+            xpp.getNamespace("prefix"); // to check - implementation is weird
+            int nsc = xpp.getNamespaceCount(xpp.getDepth());
+            if (nsc > 0) {
+                xpp.getNamespacePrefix(0/*pos*/);
+                xpp.getNamespaceUri(0/*pos*/);
+            }
+            xpp.getText(); // check handling for non START/END_TAG
+            xpp.getTextCharacters(new int[2]); // check handling for non START/END_TAG
+            switch (eventType) {
+                case MXParser.START_DOCUMENT -> {
+                    System.out.println("START_DOCUMENT");
+                }
+                case MXParser.START_TAG -> {
+                    xpp.getText(); // never uses org.apache.camel.xml.io.MXParser#pc
+                    xpp.getTextCharacters(new int[2]);
+                    xpp.isEmptyElementTag();
+                    System.out.println("START_TAG" + (xpp.isEmptyElementTag() ? " (empty tag)" : ""));
+                    System.out.println(" - name: " + xpp.getName());
+                    System.out.println(" - ns: " + xpp.getNamespace());
+                    System.out.println(" - prefix: " + xpp.getPrefix());
+                    int ac = xpp.getAttributeCount();
+                    if (ac > 0) {
+                        System.out.println(" - attributes:");
+                        for (int i = 0; i < ac; i++) {
+                            System.out.print("    - " + xpp.getAttributeName(i)
+                                             + (xpp.getAttributePrefix(i) == null
+                                                     ? "" : " (prefix: " + xpp.getAttributePrefix(i) + ")"));
+                            System.out.print(": " + xpp.getAttributeValue(i));
+                            System.out.print(", ns: " + xpp.getAttributeNamespace(i));
+                            System.out.println();
+                        }
+                    }
+                    if ("e1".equals(xpp.getName())) {
+                        assertEquals("value-1", xpp.getAttributeValue("", "a"));
+                        assertEquals("value-1", xpp.getAttributeValue(null, "a"));
+                        assertEquals("value-2", xpp.getAttributeValue("uri:b", "a"));
+                        // check with non-interned String
+                        assertEquals("value-2", xpp.getAttributeValue(new String("uri:b"), "a"));
+                    }
+                }
+                case MXParser.END_TAG -> {
+                    xpp.getText(); // never uses org.apache.camel.xml.io.MXParser#pc
+                    xpp.getTextCharacters(new int[2]);
+                    System.out.println("END_TAG");
+                    System.out.println(" - name: " + xpp.getName());
+                    System.out.println(" - ns: " + xpp.getNamespace());
+                    System.out.println(" - prefix: " + xpp.getPrefix());
+                }
+                case MXParser.TEXT -> {
+                    System.out.println("TEXT");
+                    System.out.println(" - name: " + xpp.getName());
+                    System.out.println(" - text: '" + xpp.getText() + (xpp.isWhitespace() ? "' (whitespace)" : "'"));
+                }
+                case MXParser.CDSECT -> {
+                    xpp.isWhitespace();
+                }
+                case MXParser.ENTITY_REF -> {
+                    xpp.getName();
+                    xpp.getText(); // always returns text - even if null
+                    System.out.println("ENTITY_REF");
+                    System.out.println(" - name: " + xpp.getName());
+                    System.out.println(" - text: " + xpp.getText());
+                }
+                case MXParser.IGNORABLE_WHITESPACE -> {
+                    xpp.isWhitespace(); // always true
+                }
+                case MXParser.PROCESSING_INSTRUCTION -> {
+                }
+                case MXParser.COMMENT -> {
+                }
+                case MXParser.DOCDECL -> {
+                }
+            }
+            eventType = xpp.next();
+        }
+    }
+
+    @Test
+    public void parseTheEdge() throws XmlPullParserException, IOException {
+        StringBuilder sb = new StringBuilder();
+        sb.append("<?xml version='1.0'?>\n");
+        sb.append("<!--\n");
+        for (int i = sb.toString().length() + 4 - 2; i < 8 * 1024; i += 4) {
+            sb.append("abc\n");
+        }
+        sb.append("-->\n");
+        sb.append("<root><child a=\"b\" /></root>\n");
+        BaseParser p = new BaseParser(new StringReader(sb.toString()));
+        MXParser xpp = p.parser;
+        int eventType = xpp.getEventType();
+        while (eventType != MXParser.END_DOCUMENT) {
+            eventType = xpp.next();
+            if (eventType == MXParser.START_TAG) {
+                System.out.println(xpp.getName());
+            }
+        }
+    }
+
+}
diff --git a/dsl/camel-jbang/camel-jbang-core/src/main/java/org/apache/camel/dsl/jbang/core/commands/Run.java b/dsl/camel-jbang/camel-jbang-core/src/main/java/org/apache/camel/dsl/jbang/core/commands/Run.java
index 353d875f9b4..2b59afbf1e1 100644
--- a/dsl/camel-jbang/camel-jbang-core/src/main/java/org/apache/camel/dsl/jbang/core/commands/Run.java
+++ b/dsl/camel-jbang/camel-jbang-core/src/main/java/org/apache/camel/dsl/jbang/core/commands/Run.java
@@ -63,6 +63,8 @@ import org.apache.camel.util.FileUtil;
 import org.apache.camel.util.IOHelper;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.StringHelper;
+import org.apache.camel.xml.io.util.XmlStreamDetector;
+import org.apache.camel.xml.io.util.XmlStreamInfo;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.core.config.Configurator;
 import picocli.CommandLine;
@@ -84,6 +86,17 @@ public class Run extends CamelCommand {
     private static final String[] ACCEPTED_FILE_EXT
             = new String[] { "java", "groovy", "js", "jsh", "kts", "xml", "yaml" };
 
+    private static final String[] ACCEPTED_XML_ROOT_ELEMENT_NAMES = new String[] {
+            "route", "routes",
+            "routeTemplate", "routeTemplates",
+            "templatedRoute", "templatedRoutes",
+            "rest", "rests",
+            "routeConfiguration", "beans"
+    };
+
+    private static final Set<String> ACCEPTED_XML_ROOT_ELEMENTS
+            = new HashSet<>(Arrays.asList(ACCEPTED_XML_ROOT_ELEMENT_NAMES));
+
     private static final String OPENAPI_GENERATED_FILE = ".camel-jbang/generated-openapi.yaml";
     private static final String CLIPBOARD_GENERATED_FILE = ".camel-jbang/generated-clipboard";
 
@@ -958,10 +971,15 @@ public class Run extends CamelCommand {
             if (!github && ("xml".equals(ext2) || "yaml".equals(ext2))) {
                 // load content into memory
                 try (FileInputStream fis = new FileInputStream(file)) {
-                    String data = IOHelper.loadText(fis);
                     if ("xml".equals(ext2)) {
-                        return data.contains("<routes") || data.contains("<routeConfiguration") || data.contains("<rests");
+                        XmlStreamDetector detector = new XmlStreamDetector(fis);
+                        XmlStreamInfo info = detector.information();
+                        if (!info.isValid()) {
+                            return false;
+                        }
+                        return ACCEPTED_XML_ROOT_ELEMENTS.contains(info.getRootElementName());
                     } else {
+                        String data = IOHelper.loadText(fis);
                         // also support Camel K integrations and Kamelet bindings
                         return data.contains("- from:") || data.contains("- route:") || data.contains("- route-configuration:")
                                 || data.contains("- rest:") || data.contains("- beans:")
diff --git a/dsl/camel-xml-io-dsl/src/main/java/org/apache/camel/dsl/xml/io/XmlRoutesBuilderLoader.java b/dsl/camel-xml-io-dsl/src/main/java/org/apache/camel/dsl/xml/io/XmlRoutesBuilderLoader.java
index 6de5c7185b6..c7ed134cddf 100644
--- a/dsl/camel-xml-io-dsl/src/main/java/org/apache/camel/dsl/xml/io/XmlRoutesBuilderLoader.java
+++ b/dsl/camel-xml-io-dsl/src/main/java/org/apache/camel/dsl/xml/io/XmlRoutesBuilderLoader.java
@@ -31,10 +31,16 @@ import org.apache.camel.spi.Resource;
 import org.apache.camel.spi.annotations.RoutesLoader;
 import org.apache.camel.support.CachedResource;
 import org.apache.camel.xml.in.ModelParser;
+import org.apache.camel.xml.io.util.XmlStreamDetector;
+import org.apache.camel.xml.io.util.XmlStreamInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @ManagedResource(description = "Managed XML RoutesBuilderLoader")
 @RoutesLoader(XmlRoutesBuilderLoader.EXTENSION)
 public class XmlRoutesBuilderLoader extends RouteBuilderLoaderSupport {
+    public static final Logger LOG = LoggerFactory.getLogger(XmlRoutesBuilderLoader.class);
+
     public static final String EXTENSION = "xml";
     public static final String NAMESPACE = "http://camel.apache.org/schema/spring";
     private static final List<String> NAMESPACES = List.of("", NAMESPACE);
@@ -54,20 +60,34 @@ public class XmlRoutesBuilderLoader extends RouteBuilderLoaderSupport {
 
             @Override
             public void configure() throws Exception {
-                // we use configure to load the routes (with namespace and without namespace)
-                for (String ns : NAMESPACES) {
-                    new ModelParser(resource, ns)
-                            .parseRouteTemplatesDefinition()
-                            .ifPresent(this::setRouteTemplateCollection);
-                    new ModelParser(resource, ns)
-                            .parseTemplatedRoutesDefinition()
-                            .ifPresent(this::setTemplatedRouteCollection);
-                    new ModelParser(resource, ns)
-                            .parseRestsDefinition()
-                            .ifPresent(this::setRestCollection);
-                    new ModelParser(resource, ns)
-                            .parseRoutesDefinition()
-                            .ifPresent(this::addRoutes);
+                // instead of parsing the document NxM times (for each namespace x root element combination),
+                // we preparse it using XmlStreamDetector and then parse it fully knowing what's inside.
+                // we could even do better, by passing already preparsed information through config file, but
+                // it's getting complicated when using multiple files.
+                XmlStreamDetector detector = new XmlStreamDetector(resource.getInputStream());
+                XmlStreamInfo xmlInfo = detector.information();
+                if (!xmlInfo.isValid()) {
+                    // should be valid, because we checked it before
+                    LOG.warn("Invalid XML document: {}", xmlInfo.getProblem().getMessage());
+                    return;
+                }
+                switch (xmlInfo.getRootElementName()) {
+                    case "routeTemplate", "routeTemplates" ->
+                        new ModelParser(resource, xmlInfo.getRootElementNamespace())
+                                .parseRouteTemplatesDefinition()
+                                .ifPresent(this::setRouteTemplateCollection);
+                    case "templatedRoutes", "templatedRoute" ->
+                        new ModelParser(resource, xmlInfo.getRootElementNamespace())
+                                .parseTemplatedRoutesDefinition()
+                                .ifPresent(this::setTemplatedRouteCollection);
+                    case "rests", "rest" ->
+                        new ModelParser(resource, xmlInfo.getRootElementNamespace())
+                                .parseRestsDefinition()
+                                .ifPresent(this::setRestCollection);
+                    case "routes", "route" ->
+                        new ModelParser(resource, xmlInfo.getRootElementNamespace())
+                                .parseRoutesDefinition()
+                                .ifPresent(this::addRoutes);
                 }
             }