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 2023/12/24 12:04:05 UTC

(camel) branch unmarshal created (now 35051dabe2d)

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

davsclaus pushed a change to branch unmarshal
in repository https://gitbox.apache.org/repos/asf/camel.git


      at 35051dabe2d CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

This branch includes the following new commits:

     new 698066615ab CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
     new 7285d10155a Polished
     new a49b254f07b CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
     new e8a4a0285a3 CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
     new b6c0b78abfa CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
     new ff108e6be38 CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
     new 0a92ec1c224 CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
     new 35051dabe2d CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

The 8 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



(camel) 02/08: Polished

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 7285d10155a4dea95c12d56d0cc5ffa5c1854819
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 11:49:18 2023 +0100

    Polished
---
 .../java/org/apache/camel/management/ManagedRouteNodePrefixIdTest.java  | 2 --
 1 file changed, 2 deletions(-)

diff --git a/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteNodePrefixIdTest.java b/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteNodePrefixIdTest.java
index 2b65f1afd99..840524f5f82 100644
--- a/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteNodePrefixIdTest.java
+++ b/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteNodePrefixIdTest.java
@@ -42,8 +42,6 @@ public class ManagedRouteNodePrefixIdTest extends ManagementTestSupport {
         Set<ObjectName> set = mbeanServer.queryNames(new ObjectName("*:type=processors,*"), null);
         assertEquals(4, set.size());
 
-        System.out.println(set);
-
         // hardcoded ids should also be prefixed
         ManagedProcessorMBean mb
                 = context.getCamelContextExtension().getContextPlugin(ManagedCamelContext.class)


(camel) 07/08: CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0a92ec1c2241e53fd3cec609ecb0f136f79050d7
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 12:38:45 2023 +0100

    CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
---
 .../camel/dataformat/flatpack/FlatpackDataFormat.java  | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)

diff --git a/components/camel-flatpack/src/main/java/org/apache/camel/dataformat/flatpack/FlatpackDataFormat.java b/components/camel-flatpack/src/main/java/org/apache/camel/dataformat/flatpack/FlatpackDataFormat.java
index c575e22cc34..1c9c09c07c9 100644
--- a/components/camel-flatpack/src/main/java/org/apache/camel/dataformat/flatpack/FlatpackDataFormat.java
+++ b/components/camel-flatpack/src/main/java/org/apache/camel/dataformat/flatpack/FlatpackDataFormat.java
@@ -22,6 +22,7 @@ import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Reader;
+import java.io.StringReader;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -110,7 +111,22 @@ public class FlatpackDataFormat extends ServiceSupport implements DataFormat, Da
 
     @Override
     public Object unmarshal(Exchange exchange, InputStream stream) throws Exception {
-        InputStreamReader reader = new InputStreamReader(stream, ExchangeHelper.getCharsetName(exchange));
+        return unmarshal(exchange, (Object) stream);
+    }
+
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
+        Reader reader;
+        if (body instanceof Reader r) {
+            reader = r;
+        } else if (body instanceof String s) {
+            reader = new StringReader(s);
+        } else {
+            // fallback to input stream
+            InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+            reader = new InputStreamReader(is, ExchangeHelper.getCharsetName(exchange));
+        }
+
         try {
             Parser parser = createParser(exchange, reader);
             DataSet dataSet = parser.parse();


(camel) 05/08: CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b6c0b78abfaf20da7afbc2b734a439f2223781f7
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 12:23:49 2023 +0100

    CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
---
 .../component/fastjson/FastjsonDataFormat.java     | 29 +++++++++++++++++++---
 1 file changed, 25 insertions(+), 4 deletions(-)

diff --git a/components/camel-fastjson/src/main/java/org/apache/camel/component/fastjson/FastjsonDataFormat.java b/components/camel-fastjson/src/main/java/org/apache/camel/component/fastjson/FastjsonDataFormat.java
index eeb0053cab1..a21a9cb4c1b 100644
--- a/components/camel-fastjson/src/main/java/org/apache/camel/component/fastjson/FastjsonDataFormat.java
+++ b/components/camel-fastjson/src/main/java/org/apache/camel/component/fastjson/FastjsonDataFormat.java
@@ -109,11 +109,32 @@ public class FastjsonDataFormat extends ServiceSupport
     }
 
     @Override
-    public Object unmarshal(final Exchange exchange, final InputStream stream) throws Exception {
-        if (unmarshalGenericType == null) {
-            return JSON.parseObject(stream, config.getCharset(), unmarshalType, config.getFeatures());
+    public Object unmarshal(Exchange exchange, InputStream stream) throws Exception {
+        return unmarshal(exchange, (Object) stream);
+    }
+
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
+        if (body instanceof String str) {
+            if (unmarshalGenericType == null) {
+                return JSON.parseObject(str, unmarshalType, config.getFeatures());
+            } else {
+                return JSON.parseObject(str, unmarshalGenericType, config.getFeatures());
+            }
+        } else if (body instanceof byte[] arr) {
+            if (unmarshalGenericType == null) {
+                return JSON.parseObject(arr, unmarshalType, config.getFeatures());
+            } else {
+                return JSON.parseObject(arr, unmarshalGenericType, config.getFeatures());
+            }
         } else {
-            return JSON.parseObject(stream, config.getCharset(), unmarshalGenericType, config.getFeatures());
+            // fallback to input stream
+            InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+            if (unmarshalGenericType == null) {
+                return JSON.parseObject(is, config.getCharset(), unmarshalType, config.getFeatures());
+            } else {
+                return JSON.parseObject(is, config.getCharset(), unmarshalGenericType, config.getFeatures());
+            }
         }
     }
 


(camel) 06/08: CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ff108e6be382738792165b981eadc4986e21aa51
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 12:34:04 2023 +0100

    CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
---
 .../camel/component/gson/GsonDataFormat.java       | 49 +++++++++++++++++-----
 .../camel/component/gson/GsonDataFormatTest.java   |  7 +---
 2 files changed, 39 insertions(+), 17 deletions(-)

diff --git a/components/camel-gson/src/main/java/org/apache/camel/component/gson/GsonDataFormat.java b/components/camel-gson/src/main/java/org/apache/camel/component/gson/GsonDataFormat.java
index e35b04b3f99..294fd30e1b1 100644
--- a/components/camel-gson/src/main/java/org/apache/camel/component/gson/GsonDataFormat.java
+++ b/components/camel-gson/src/main/java/org/apache/camel/component/gson/GsonDataFormat.java
@@ -16,12 +16,12 @@
  */
 package org.apache.camel.component.gson;
 
-import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
+import java.io.Reader;
 import java.lang.reflect.Type;
 import java.util.Arrays;
 import java.util.List;
@@ -32,6 +32,7 @@ import com.google.gson.FieldNamingStrategy;
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.LongSerializationPolicy;
+import com.google.gson.stream.JsonReader;
 import org.apache.camel.CamelContext;
 import org.apache.camel.CamelContextAware;
 import org.apache.camel.Exchange;
@@ -152,17 +153,43 @@ public class GsonDataFormat extends ServiceSupport
 
     @Override
     public Object unmarshal(final Exchange exchange, final InputStream stream) throws Exception {
-        try (final InputStreamReader isr = new InputStreamReader(stream, ExchangeHelper.getCharsetName(exchange));
-             final BufferedReader reader = IOHelper.buffered(isr)) {
-
-            String type = exchange.getIn().getHeader(GsonConstants.UNMARSHAL_TYPE, String.class);
-            if (type != null) {
-                Class<?> clazz = exchange.getContext().getClassResolver().resolveMandatoryClass(type);
-                return gson.fromJson(reader, clazz);
-            } else if (unmarshalGenericType == null) {
-                return gson.fromJson(reader, unmarshalType);
+        return unmarshal(exchange, (Object) stream);
+    }
+
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
+        Class<?> clazz = unmarshalType;
+        String type = exchange.getIn().getHeader(GsonConstants.UNMARSHAL_TYPE, String.class);
+        if (type != null) {
+            clazz = exchange.getContext().getClassResolver().resolveMandatoryClass(type);
+        }
+
+        if (body instanceof String str) {
+            if (unmarshalGenericType == null) {
+                return gson.fromJson(str, clazz);
+            } else {
+                return gson.fromJson(str, unmarshalGenericType);
+            }
+        } else if (body instanceof Reader r) {
+            if (unmarshalGenericType == null) {
+                return gson.fromJson(r, clazz);
+            } else {
+                return gson.fromJson(r, unmarshalGenericType);
+            }
+        } else if (body instanceof JsonReader r) {
+            if (unmarshalGenericType == null) {
+                return gson.fromJson(r, clazz);
+            } else {
+                return gson.fromJson(r, unmarshalGenericType);
+            }
+        } else {
+            // fallback to input stream
+            InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+            Reader r = new InputStreamReader(is);
+            if (unmarshalGenericType == null) {
+                return gson.fromJson(r, clazz);
             } else {
-                return gson.fromJson(reader, unmarshalGenericType);
+                return gson.fromJson(r, unmarshalGenericType);
             }
         }
     }
diff --git a/components/camel-gson/src/test/java/org/apache/camel/component/gson/GsonDataFormatTest.java b/components/camel-gson/src/test/java/org/apache/camel/component/gson/GsonDataFormatTest.java
index dfc17e89e0b..8aedcd8608a 100644
--- a/components/camel-gson/src/test/java/org/apache/camel/component/gson/GsonDataFormatTest.java
+++ b/components/camel-gson/src/test/java/org/apache/camel/component/gson/GsonDataFormatTest.java
@@ -16,8 +16,6 @@
  */
 package org.apache.camel.component.gson;
 
-import java.io.*;
-import java.nio.charset.StandardCharsets;
 import java.util.*;
 
 import org.apache.camel.Exchange;
@@ -41,7 +39,6 @@ public class GsonDataFormatTest {
 
     @BeforeEach
     public void setup() {
-        when(message.getHeader(Exchange.CHARSET_NAME, String.class)).thenReturn(StandardCharsets.UTF_8.name());
         when(exchange.getIn()).thenReturn(message);
     }
 
@@ -64,9 +61,7 @@ public class GsonDataFormatTest {
         Object unmarshalled;
         try (GsonDataFormat gsonDataFormat = new GsonDataFormat()) {
             gsonDataFormat.doStart();
-            try (InputStream in = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8))) {
-                unmarshalled = gsonDataFormat.unmarshal(exchange, in);
-            }
+            unmarshalled = gsonDataFormat.unmarshal(exchange, json);
             assertEquals(expected, unmarshalled);
         }
     }


(camel) 04/08: CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e8a4a0285a31778c168488817ea39a09fa5e4e92
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 12:14:52 2023 +0100

    CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
---
 .../apache/camel/dataformat/csv/CsvDataFormat.java |  7 ++++-
 .../camel/dataformat/csv/CsvUnmarshaller.java      | 30 ++++++++++++++--------
 2 files changed, 26 insertions(+), 11 deletions(-)

diff --git a/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvDataFormat.java b/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvDataFormat.java
index 3de23c32eca..9d400e7e6b1 100644
--- a/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvDataFormat.java
+++ b/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvDataFormat.java
@@ -92,7 +92,12 @@ public class CsvDataFormat extends ServiceSupport implements DataFormat, DataFor
 
     @Override
     public Object unmarshal(Exchange exchange, InputStream inputStream) throws Exception {
-        return unmarshaller.unmarshal(exchange, inputStream);
+        return unmarshal(exchange, (Object) inputStream);
+    }
+
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
+        return unmarshaller.unmarshal(exchange, body);
     }
 
     @Override
diff --git a/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvUnmarshaller.java b/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvUnmarshaller.java
index 060537730ca..6811ef71166 100644
--- a/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvUnmarshaller.java
+++ b/components/camel-csv/src/main/java/org/apache/camel/dataformat/csv/CsvUnmarshaller.java
@@ -69,12 +69,12 @@ abstract class CsvUnmarshaller {
     /**
      * Unmarshal the CSV
      *
-     * @param  exchange    Exchange (used for accessing type converter)
-     * @param  inputStream Input CSV stream
-     * @return             Unmarshalled CSV
-     * @throws IOException if the stream cannot be read properly
+     * @param  exchange  Exchange (used for accessing type converter)
+     * @param  body      the input
+     * @return           Unmarshalled CSV
+     * @throws Exception if error during unmarshalling
      */
-    public abstract Object unmarshal(Exchange exchange, InputStream inputStream) throws IOException;
+    public abstract Object unmarshal(Exchange exchange, Object body) throws Exception;
 
     private static CsvRecordConverter<?> extractConverter(CsvDataFormat dataFormat) {
         if (dataFormat.getRecordConverter() != null) {
@@ -99,9 +99,15 @@ abstract class CsvUnmarshaller {
         }
 
         @Override
-        public Object unmarshal(Exchange exchange, InputStream inputStream) throws IOException {
+        public Object unmarshal(Exchange exchange, Object body) throws Exception {
+            Reader reader = exchange.getContext().getTypeConverter().tryConvertTo(Reader.class, exchange, body);
+            if (reader == null) {
+                // fallback to input stream
+                InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+                reader = new InputStreamReader(is, ExchangeHelper.getCharsetName(exchange));
+            }
             CSVParser parser
-                    = new CSVParser(new InputStreamReader(inputStream, ExchangeHelper.getCharsetName(exchange)), format);
+                    = new CSVParser(reader, format);
             try {
                 if (dataFormat.isCaptureHeaderRecord()) {
                     exchange.getMessage().setHeader(CsvConstants.HEADER_RECORD, parser.getHeaderNames());
@@ -132,10 +138,14 @@ abstract class CsvUnmarshaller {
         }
 
         @Override
-        public Object unmarshal(Exchange exchange, InputStream inputStream) throws IOException {
-            Reader reader = null;
+        public Object unmarshal(Exchange exchange, Object body) throws Exception {
+            Reader reader = exchange.getContext().getTypeConverter().tryConvertTo(Reader.class, exchange, body);
+            if (reader == null) {
+                // fallback to input stream
+                InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+                reader = new InputStreamReader(is, ExchangeHelper.getCharsetName(exchange));
+            }
             try {
-                reader = new InputStreamReader(inputStream, ExchangeHelper.getCharsetName(exchange));
                 CSVParser parser = new CSVParser(reader, format);
                 CsvIterator<?> answer = new CsvIterator<>(parser, converter);
                 // add to UoW, so we can close the iterator, so it can release any resources


(camel) 01/08: CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 698066615ab2e01f9b3aa98693d763f3d9f6d7d4
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 11:49:03 2023 +0100

    CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
---
 .../camel/converter/jaxb/JaxbDataFormat.java       | 34 ++++++++++++++++------
 .../main/java/org/apache/camel/spi/DataFormat.java | 28 ++++++++++++++++++
 .../org/apache/camel/model/SplitDefinition.java    |  7 ++---
 .../transformer/DataFormatTransformer.java         |  4 +--
 .../rest/RestProducerBindingProcessorTest.java     |  4 +--
 .../camel/processor/SplitterSingleMapTest.java     |  6 ++--
 .../camel/processor/converter/ConvertBodyTest.java |  4 ---
 .../processor/converter/ConvertHeaderTest.java     |  4 ---
 .../support/processor/UnmarshalProcessor.java      |  4 +--
 9 files changed, 64 insertions(+), 31 deletions(-)

diff --git a/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java b/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
index dcb0e2c0db6..7bb3db0c02f 100644
--- a/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
+++ b/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
-import java.io.UnsupportedEncodingException;
+import java.io.Reader;
 import java.lang.reflect.Method;
 import java.net.MalformedURLException;
 import java.net.URL;
@@ -53,12 +53,12 @@ import org.apache.camel.CamelContextAware;
 import org.apache.camel.Exchange;
 import org.apache.camel.ExchangePropertyKey;
 import org.apache.camel.InvalidPayloadException;
+import org.apache.camel.NoTypeConversionAvailableException;
 import org.apache.camel.TypeConverter;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.DataFormatContentTypeHeader;
 import org.apache.camel.spi.DataFormatName;
 import org.apache.camel.spi.annotations.Dataformat;
-import org.apache.camel.support.ExchangeHelper;
 import org.apache.camel.support.ResourceHelper;
 import org.apache.camel.support.service.ServiceSupport;
 import org.apache.camel.util.IOHelper;
@@ -271,16 +271,26 @@ public class JaxbDataFormat extends ServiceSupport
     }
 
     @Override
-    public Object unmarshal(Exchange exchange, InputStream stream) throws IOException {
+    public Object unmarshal(Exchange exchange, InputStream stream) throws Exception {
+        throw new UnsupportedOperationException("Not in use");
+    }
+
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
         try {
             Object answer;
 
-            final XMLStreamReader xmlReader;
+            XMLStreamReader xmlReader;
             if (needFiltering(exchange)) {
                 xmlReader
-                        = typeConverter.convertTo(XMLStreamReader.class, exchange, createNonXmlFilterReader(exchange, stream));
+                        = typeConverter.convertTo(XMLStreamReader.class, exchange, createNonXmlFilterReader(exchange, body));
             } else {
-                xmlReader = typeConverter.convertTo(XMLStreamReader.class, exchange, stream);
+                xmlReader = typeConverter.tryConvertTo(XMLStreamReader.class, exchange, body);
+                if (xmlReader == null) {
+                    // fallback to input stream
+                    InputStream is = getCamelContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+                    xmlReader = typeConverter.convertTo(XMLStreamReader.class, exchange, is);
+                }
             }
             String partClassFromHeader = exchange.getIn().getHeader(JaxbConstants.JAXB_PART_CLASS, String.class);
             if (partClass != null || partClassFromHeader != null) {
@@ -306,9 +316,15 @@ public class JaxbDataFormat extends ServiceSupport
         }
     }
 
-    private NonXmlFilterReader createNonXmlFilterReader(Exchange exchange, InputStream stream)
-            throws UnsupportedEncodingException {
-        return new NonXmlFilterReader(new InputStreamReader(stream, ExchangeHelper.getCharsetName(exchange)));
+    private NonXmlFilterReader createNonXmlFilterReader(Exchange exchange, Object body)
+            throws NoTypeConversionAvailableException {
+        Reader reader = getCamelContext().getTypeConverter().tryConvertTo(Reader.class, exchange, body);
+        if (reader == null) {
+            // fallback to input stream
+            InputStream is = getCamelContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+            reader = new InputStreamReader(is);
+        }
+        return new NonXmlFilterReader(reader);
     }
 
     protected boolean needFiltering(Exchange exchange) {
diff --git a/core/camel-api/src/main/java/org/apache/camel/spi/DataFormat.java b/core/camel-api/src/main/java/org/apache/camel/spi/DataFormat.java
index 78ddc9ab059..c461b8ad97e 100644
--- a/core/camel-api/src/main/java/org/apache/camel/spi/DataFormat.java
+++ b/core/camel-api/src/main/java/org/apache/camel/spi/DataFormat.java
@@ -52,6 +52,34 @@ public interface DataFormat extends Service {
      * @param  stream    the input stream with the object to be unmarshalled
      * @return           the unmarshalled object
      * @throws Exception can be thrown
+     * @see              #unmarshal(Exchange, Object)
      */
     Object unmarshal(Exchange exchange, InputStream stream) throws Exception;
+
+    /**
+     * Unmarshals the given body into an object.
+     * <p/>
+     * <b>Notice:</b> The result is set as body on the exchange OUT message. It is possible to mutate the OUT message
+     * provided in the given exchange parameter. For instance adding headers to the OUT message will be preserved.
+     * <p/>
+     * It's also legal to return the <b>same</b> passed <tt>exchange</tt> as is but also a {@link Message} object as
+     * well which will be used as the OUT message of <tt>exchange</tt>.
+     * <p/>
+     * This method can be used when a dataformat is optimized to handle any kind of message body as-is. For example
+     * camel-jaxb has been optimized to do this. The regular {@link #unmarshal(Exchange, InputStream)} method requires
+     * Camel to convert the message body into an {@link InputStream} prior to calling the unmarshal method. This can be
+     * avoided if the data-format implementation can be optimized to handle this by itself, such as camel-jaxb that can
+     * handle message body as a String payload out of the box. When a data format implementation is using this method,
+     * then the {@link #unmarshal(Exchange, InputStream)} must also be implemented but should be empty, as Camel will
+     * not invoke this method.
+     *
+     * @param  exchange  the current exchange
+     * @param  body      the input object to be unmarshalled
+     * @return           the unmarshalled object
+     * @throws Exception can be thrown
+     */
+    default Object unmarshal(Exchange exchange, Object body) throws Exception {
+        InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+        return unmarshal(exchange, is);
+    }
 }
diff --git a/core/camel-core-model/src/main/java/org/apache/camel/model/SplitDefinition.java b/core/camel-core-model/src/main/java/org/apache/camel/model/SplitDefinition.java
index de0b40653d2..a323920c93f 100644
--- a/core/camel-core-model/src/main/java/org/apache/camel/model/SplitDefinition.java
+++ b/core/camel-core-model/src/main/java/org/apache/camel/model/SplitDefinition.java
@@ -115,10 +115,9 @@ public class SplitDefinition extends OutputExpressionNode implements ExecutorSer
     // -------------------------------------------------------------------------
 
     /**
-     * Delimiter used in splitting messages.
-     * Can be turned off using the value <tt>false</tt>.
-     * To force not splitting then the delimiter can be set to <tt>single</tt> to use the value as a single list,
-     * this can be needed in some special situations.
+     * Delimiter used in splitting messages. Can be turned off using the value <tt>false</tt>. To force not splitting
+     * then the delimiter can be set to <tt>single</tt> to use the value as a single list, this can be needed in some
+     * special situations.
      * <p/>
      * The default value is comma.
      *
diff --git a/core/camel-core-processor/src/main/java/org/apache/camel/processor/transformer/DataFormatTransformer.java b/core/camel-core-processor/src/main/java/org/apache/camel/processor/transformer/DataFormatTransformer.java
index 7e197d7c942..c58d156a59f 100644
--- a/core/camel-core-processor/src/main/java/org/apache/camel/processor/transformer/DataFormatTransformer.java
+++ b/core/camel-core-processor/src/main/java/org/apache/camel/processor/transformer/DataFormatTransformer.java
@@ -16,8 +16,6 @@
  */
 package org.apache.camel.processor.transformer;
 
-import java.io.InputStream;
-
 import org.apache.camel.CamelContext;
 import org.apache.camel.Exchange;
 import org.apache.camel.Message;
@@ -64,7 +62,7 @@ public class DataFormatTransformer extends Transformer {
         // Unmarshaling into Java Object
         if ((DataType.isAnyType(to) || to.isJavaType()) && (from.equals(getFrom()) || from.getScheme().equals(getName()))) {
             LOG.debug("Unmarshaling with: {}", dataFormat);
-            Object answer = dataFormat.unmarshal(exchange, message.getBody(InputStream.class));
+            Object answer = dataFormat.unmarshal(exchange, message.getBody());
             if (!DataType.isAnyType(to) && to.getName() != null) {
                 Class<?> toClass = context.getClassResolver().resolveClass(to.getName());
                 if (!toClass.isAssignableFrom(answer.getClass())) {
diff --git a/core/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java b/core/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java
index 4b6400bb25d..e6d345d5409 100644
--- a/core/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java
+++ b/core/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java
@@ -70,7 +70,7 @@ public class RestProducerBindingProcessorTest {
         exchange.setIn(input);
 
         final ResponsePojo response = new ResponsePojo();
-        when(outJsonDataFormat.unmarshal(same(exchange), any(InputStream.class))).thenReturn(response);
+        when(outJsonDataFormat.unmarshal(same(exchange), any(Object.class))).thenReturn(response);
 
         final ArgumentCaptor<AsyncCallback> bindingCallback = ArgumentCaptor.forClass(AsyncCallback.class);
 
@@ -105,7 +105,7 @@ public class RestProducerBindingProcessorTest {
         exchange.setIn(input);
 
         final ResponsePojo response = new ResponsePojo();
-        when(outXmlDataFormat.unmarshal(same(exchange), any(InputStream.class))).thenReturn(response);
+        when(outXmlDataFormat.unmarshal(same(exchange), any(Object.class))).thenReturn(response);
 
         final ArgumentCaptor<AsyncCallback> bindingCallback = ArgumentCaptor.forClass(AsyncCallback.class);
 
diff --git a/core/camel-core/src/test/java/org/apache/camel/processor/SplitterSingleMapTest.java b/core/camel-core/src/test/java/org/apache/camel/processor/SplitterSingleMapTest.java
index ff12d7c2b55..a47b7434e84 100644
--- a/core/camel-core/src/test/java/org/apache/camel/processor/SplitterSingleMapTest.java
+++ b/core/camel-core/src/test/java/org/apache/camel/processor/SplitterSingleMapTest.java
@@ -16,14 +16,14 @@
  */
 package org.apache.camel.processor;
 
+import java.util.LinkedHashMap;
+import java.util.Map;
+
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
 import org.junit.jupiter.api.Test;
 
-import java.util.LinkedHashMap;
-import java.util.Map;
-
 public class SplitterSingleMapTest extends ContextTestSupport {
 
     @Test
diff --git a/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertBodyTest.java b/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertBodyTest.java
index c7af7ffbd0c..af5f8745510 100644
--- a/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertBodyTest.java
+++ b/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertBodyTest.java
@@ -33,10 +33,6 @@ import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.condition.DisabledOnOs;
 import org.junit.jupiter.api.condition.OS;
 
-import java.io.ByteArrayInputStream;
-import java.nio.charset.UnsupportedCharsetException;
-import java.util.Date;
-
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
diff --git a/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertHeaderTest.java b/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertHeaderTest.java
index 799547701e3..c3a5be38fcc 100644
--- a/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertHeaderTest.java
+++ b/core/camel-core/src/test/java/org/apache/camel/processor/converter/ConvertHeaderTest.java
@@ -31,10 +31,6 @@ import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.condition.DisabledOnOs;
 import org.junit.jupiter.api.condition.OS;
 
-import java.io.ByteArrayInputStream;
-import java.nio.charset.UnsupportedCharsetException;
-import java.util.Date;
-
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/processor/UnmarshalProcessor.java b/core/camel-support/src/main/java/org/apache/camel/support/processor/UnmarshalProcessor.java
index 8b710a64bb0..25e2acba8ac 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/processor/UnmarshalProcessor.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/processor/UnmarshalProcessor.java
@@ -67,13 +67,13 @@ public class UnmarshalProcessor extends AsyncProcessorSupport implements Traceab
                 // The body is null, and it is an allowed value so let's skip the unmarshalling
                 out = exchange.getOut();
             } else {
-                stream = in.getMandatoryBody(InputStream.class);
+                Object body = in.getBody();
 
                 // lets set up the out message before we invoke the dataFormat so that it can mutate it if necessary
                 out = exchange.getOut();
                 out.copyFrom(in);
 
-                result = dataFormat.unmarshal(exchange, stream);
+                result = dataFormat.unmarshal(exchange, body);
             }
             if (result instanceof Exchange) {
                 if (result != exchange) {


(camel) 03/08: CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a49b254f07bbbf3a7ee5011bd530217f1f63450a
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 12:08:13 2023 +0100

    CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
---
 .../jackson/AbstractJacksonDataFormat.java         | 40 ++++++++++++++++++++--
 .../component/jacksonxml/JacksonXMLDataFormat.java | 38 ++++++++++++++++++--
 .../camel/converter/jaxb/JaxbDataFormat.java       |  2 +-
 3 files changed, 75 insertions(+), 5 deletions(-)

diff --git a/components/camel-jackson/src/main/java/org/apache/camel/component/jackson/AbstractJacksonDataFormat.java b/components/camel-jackson/src/main/java/org/apache/camel/component/jackson/AbstractJacksonDataFormat.java
index 7bf41509fa0..a1d686da2b7 100644
--- a/components/camel-jackson/src/main/java/org/apache/camel/component/jackson/AbstractJacksonDataFormat.java
+++ b/components/camel-jackson/src/main/java/org/apache/camel/component/jackson/AbstractJacksonDataFormat.java
@@ -16,8 +16,10 @@
  */
 package org.apache.camel.component.jackson;
 
+import java.io.File;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.io.Reader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -30,9 +32,11 @@ import java.util.TimeZone;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.core.FormatSchema;
 import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.MapperFeature;
 import com.fasterxml.jackson.databind.Module;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.PropertyNamingStrategies;
 import com.fasterxml.jackson.databind.PropertyNamingStrategy;
 import com.fasterxml.jackson.databind.SerializationFeature;
@@ -40,6 +44,7 @@ import com.fasterxml.jackson.databind.type.CollectionType;
 import org.apache.camel.CamelContext;
 import org.apache.camel.CamelContextAware;
 import org.apache.camel.Exchange;
+import org.apache.camel.WrappedFile;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.DataFormatContentTypeHeader;
 import org.apache.camel.spi.DataFormatName;
@@ -161,6 +166,11 @@ public abstract class AbstractJacksonDataFormat extends ServiceSupport
 
     @Override
     public Object unmarshal(Exchange exchange, InputStream stream) throws Exception {
+        return unmarshal(exchange, (Object) stream);
+    }
+
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
         FormatSchema schema = null;
         if (this.schemaResolver != null) {
             schema = this.schemaResolver.resolve(exchange);
@@ -177,12 +187,38 @@ public abstract class AbstractJacksonDataFormat extends ServiceSupport
         if (type != null) {
             clazz = exchange.getContext().getClassResolver().resolveMandatoryClass(type);
         }
+
+        ObjectReader reader;
         if (collectionType != null) {
             CollectionType collType = objectMapper.getTypeFactory().constructCollectionType(collectionType, clazz);
-            return this.objectMapper.readerFor(collType).with(schema).readValue(stream);
+            reader = this.objectMapper.readerFor(collType).with(schema);
+        } else {
+            reader = this.objectMapper.reader(schema).forType(clazz);
+        }
+
+        // unwrap file (such as from camel-file)
+        if (body instanceof WrappedFile<?>) {
+            body = ((WrappedFile<?>) body).getBody();
+        }
+        Object answer;
+        if (body instanceof String b) {
+            answer = reader.readValue(b);
+        } else if (body instanceof byte[] arr) {
+            answer = reader.readValue(arr);
+        } else if (body instanceof Reader r) {
+            answer = reader.readValue(r);
+        } else if (body instanceof File f) {
+            answer = reader.readValue(f);
+        } else if (body instanceof JsonNode n) {
+            answer = reader.readValue(n);
         } else {
-            return this.objectMapper.reader(schema).readValue(stream, clazz);
+            // fallback to input stream
+            InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+            answer = reader.readValue(is);
         }
+
+        return answer;
+
     }
 
     // Properties
diff --git a/components/camel-jacksonxml/src/main/java/org/apache/camel/component/jacksonxml/JacksonXMLDataFormat.java b/components/camel-jacksonxml/src/main/java/org/apache/camel/component/jacksonxml/JacksonXMLDataFormat.java
index 18192f32afb..4ec59b5bf8f 100644
--- a/components/camel-jacksonxml/src/main/java/org/apache/camel/component/jacksonxml/JacksonXMLDataFormat.java
+++ b/components/camel-jacksonxml/src/main/java/org/apache/camel/component/jacksonxml/JacksonXMLDataFormat.java
@@ -16,8 +16,10 @@
  */
 package org.apache.camel.component.jacksonxml;
 
+import java.io.File;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.io.Reader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -28,8 +30,10 @@ import java.util.TimeZone;
 
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.MapperFeature;
 import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.SerializationFeature;
 import com.fasterxml.jackson.databind.type.CollectionType;
 import com.fasterxml.jackson.dataformat.xml.XmlMapper;
@@ -38,6 +42,7 @@ import com.fasterxml.jackson.module.jakarta.xmlbind.JakartaXmlBindAnnotationModu
 import org.apache.camel.CamelContext;
 import org.apache.camel.CamelContextAware;
 import org.apache.camel.Exchange;
+import org.apache.camel.WrappedFile;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.DataFormatContentTypeHeader;
 import org.apache.camel.spi.DataFormatName;
@@ -173,7 +178,11 @@ public class JacksonXMLDataFormat extends ServiceSupport
 
     @Override
     public Object unmarshal(Exchange exchange, InputStream stream) throws Exception {
+        return unmarshal(exchange, (Object) stream);
+    }
 
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
         // is there a header with the unmarshal type?
         Class<?> clazz = unmarshalType;
         String type = null;
@@ -186,12 +195,37 @@ public class JacksonXMLDataFormat extends ServiceSupport
         if (type != null) {
             clazz = exchange.getContext().getClassResolver().resolveMandatoryClass(type);
         }
+
+        ObjectReader reader;
         if (collectionType != null) {
             CollectionType collType = xmlMapper.getTypeFactory().constructCollectionType(collectionType, clazz);
-            return this.xmlMapper.readValue(stream, collType);
+            reader = this.xmlMapper.readerFor(collType);
+        } else {
+            reader = this.xmlMapper.reader().forType(clazz);
+        }
+
+        // unwrap file (such as from camel-file)
+        if (body instanceof WrappedFile<?>) {
+            body = ((WrappedFile<?>) body).getBody();
+        }
+        Object answer;
+        if (body instanceof String b) {
+            answer = reader.readValue(b);
+        } else if (body instanceof byte[] arr) {
+            answer = reader.readValue(arr);
+        } else if (body instanceof Reader r) {
+            answer = reader.readValue(r);
+        } else if (body instanceof File f) {
+            answer = reader.readValue(f);
+        } else if (body instanceof JsonNode n) {
+            answer = reader.readValue(n);
         } else {
-            return this.xmlMapper.readValue(stream, clazz);
+            // fallback to input stream
+            InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+            answer = reader.readValue(is);
         }
+
+        return answer;
     }
 
     // Properties
diff --git a/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java b/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
index 7bb3db0c02f..dbd845c19c6 100644
--- a/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
+++ b/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
@@ -272,7 +272,7 @@ public class JaxbDataFormat extends ServiceSupport
 
     @Override
     public Object unmarshal(Exchange exchange, InputStream stream) throws Exception {
-        throw new UnsupportedOperationException("Not in use");
+        return unmarshal(exchange, (Object) stream);
     }
 
     @Override


(camel) 08/08: CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 35051dabe2df12b8a371cbce6556e61efb3aa1d5
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 24 12:43:29 2023 +0100

    CAMEL-14028: Allow DataFormats to unmarshal known data formats without first converting to bytes
---
 .../camel/component/grok/GrokDataFormat.java       | 22 +++++++++++++++++++---
 1 file changed, 19 insertions(+), 3 deletions(-)

diff --git a/components/camel-grok/src/main/java/org/apache/camel/component/grok/GrokDataFormat.java b/components/camel-grok/src/main/java/org/apache/camel/component/grok/GrokDataFormat.java
index fd0a4f8c196..8d47841e820 100644
--- a/components/camel-grok/src/main/java/org/apache/camel/component/grok/GrokDataFormat.java
+++ b/components/camel-grok/src/main/java/org/apache/camel/component/grok/GrokDataFormat.java
@@ -20,6 +20,8 @@ import java.io.BufferedReader;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
+import java.io.Reader;
+import java.io.StringReader;
 import java.nio.CharBuffer;
 import java.util.*;
 import java.util.stream.Stream;
@@ -131,17 +133,31 @@ public class GrokDataFormat extends ServiceSupport implements DataFormat, DataFo
 
     @Override
     public Object unmarshal(Exchange exchange, InputStream stream) throws Exception {
+        return unmarshal(exchange, (Object) stream);
+    }
+
+    @Override
+    public Object unmarshal(Exchange exchange, Object body) throws Exception {
         List<Map<String, Object>> result = new ArrayList<>();
 
-        InputStreamReader in = new InputStreamReader(stream, ExchangeHelper.getCharsetName(exchange));
-        try (Stream<String> lines = new BufferedReader(in).lines()) {
+        Reader reader = null;
+        if (body instanceof String s) {
+            reader = new StringReader(s);
+        } else if (body instanceof Reader r) {
+            reader = r;
+        } else {
+            // fallback to input stream
+            InputStream is = exchange.getContext().getTypeConverter().mandatoryConvertTo(InputStream.class, exchange, body);
+            reader = new InputStreamReader(is, ExchangeHelper.getCharsetName(exchange));
+        }
+
+        try (Stream<String> lines = new BufferedReader(reader).lines()) {
             lines.forEachOrdered(line -> processLine(line, result));
         }
 
         if (result.isEmpty()) {
             return null;
         }
-
         if (result.size() == 1) {
             return result.get(0);
         }