You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by fo...@apache.org on 2015/07/18 08:45:22 UTC

camel git commit: jsonpath: automatic charset detection

Repository: camel
Updated Branches:
  refs/heads/master efcda65ea -> 604d2d8c2


jsonpath: automatic charset detection

Change-Id: Ic06747da9bce8a3ff43198f01c558cdc146b3a48


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

Branch: refs/heads/master
Commit: 604d2d8c2f5b6176608f7dad827237f465980ac4
Parents: efcda65
Author: Franz Forsthofer <fr...@sap.com>
Authored: Fri Jul 3 08:34:57 2015 +0200
Committer: Franz Forsthofer <fr...@sap.com>
Committed: Sat Jul 18 08:43:34 2015 +0200

----------------------------------------------------------------------
 .../camel/jsonpath/JsonPathConstants.java       |  29 ++
 .../apache/camel/jsonpath/JsonPathEngine.java   |  44 ++-
 .../org/apache/camel/jsonpath/JsonStream.java   | 292 +++++++++++++++++++
 .../camel/jsonpath/JsonPathCharsetTest.java     | 112 +++++++
 .../apache/camel/jsonpath/JsonStreamTest.java   | 130 +++++++++
 .../src/test/resources/booksUTF16BE.json        | Bin 0 -> 1106 bytes
 .../src/test/resources/booksUTF16LE.json        | Bin 0 -> 1106 bytes
 .../src/test/resources/json_stream/fourChar     |   1 +
 .../resources/json_stream/jsonISO8859-1.txt     |   1 +
 .../json_stream/jsonUCS2BigEndianWithBOM.txt    | Bin 0 -> 238 bytes
 .../json_stream/jsonUCS2BigEndianWithoutBOM.txt | Bin 0 -> 236 bytes
 .../json_stream/jsonUCS2LittleEndianWithBom.txt | Bin 0 -> 238 bytes
 .../jsonUCS2LittleEndianWithoutBOM.txt          | Bin 0 -> 236 bytes
 .../json_stream/jsonUTF32BEWithBOM.txt          | Bin 0 -> 476 bytes
 .../json_stream/jsonUTF32BEWithoutBOM.txt       | Bin 0 -> 472 bytes
 .../json_stream/jsonUTF32LEWithBOM.txt          | Bin 0 -> 476 bytes
 .../json_stream/jsonUTF32LEWithoutBOM.txt       | Bin 0 -> 472 bytes
 .../src/test/resources/json_stream/jsonUTF8.txt |   1 +
 .../json_stream/jsonUTF8WithoutBOM.txt          |   1 +
 .../src/test/resources/json_stream/oneChar      |   1 +
 20 files changed, 587 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathConstants.java
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathConstants.java b/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathConstants.java
new file mode 100644
index 0000000..9f2aa2a
--- /dev/null
+++ b/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathConstants.java
@@ -0,0 +1,29 @@
+/**
+ * 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.jsonpath;
+
+public final class JsonPathConstants {
+    
+    /** Header name for the encoding of the JSON document. */
+    public static final String HEADER_JSON_ENCODING = "CamelJsonPathJsonEncoding";
+
+    private JsonPathConstants() {
+        // no instance
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathEngine.java
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathEngine.java b/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathEngine.java
index 363492d..941aa96 100644
--- a/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathEngine.java
+++ b/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonPathEngine.java
@@ -17,22 +17,19 @@
 package org.apache.camel.jsonpath;
 
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.URL;
-import java.nio.charset.Charset;
 
 import com.jayway.jsonpath.Configuration;
 import com.jayway.jsonpath.Configuration.Defaults;
 import com.jayway.jsonpath.JsonPath;
 import com.jayway.jsonpath.Option;
 import com.jayway.jsonpath.internal.DefaultsImpl;
+
 import org.apache.camel.Exchange;
 import org.apache.camel.InvalidPayloadException;
-import org.apache.camel.NoTypeConversionAvailableException;
-import org.apache.camel.WrappedFile;
 import org.apache.camel.component.file.GenericFile;
-import org.apache.camel.component.file.GenericFileConverter;
 
 public class JsonPathEngine {
 
@@ -65,32 +62,29 @@ public class JsonPathEngine {
         Object json = exchange.getIn().getBody();
 
         if (json instanceof GenericFile) {
-            try {
-                json = GenericFileConverter.genericFileToInputStream((GenericFile<?>) json, exchange);
-            } catch (NoTypeConversionAvailableException e) {
-                json = ((WrappedFile<?>) json).getFile();
+            GenericFile<?> genericFile = (GenericFile<?>) json;
+            if (genericFile.getCharset() != null) {
+                // special treatment for generic file with charset
+                InputStream inputStream = new FileInputStream((File) genericFile.getFile());
+                return path.read(inputStream, genericFile.getCharset(), configuration);
             }
-        } else if (json instanceof WrappedFile) {
-            json = ((WrappedFile<?>) json).getFile();
         }
 
-        // the message body type should use the suitable read method
         if (json instanceof String) {
             String str = (String) json;
             return path.read(str, configuration);
-        } else if (json instanceof InputStream) {
-            InputStream is = (InputStream) json;
-            return path.read(is, Charset.defaultCharset().displayName(), configuration);
-        } else if (json instanceof File) {
-            File file = (File) json;
-            return path.read(file, configuration);
-        } else if (json instanceof URL) {
-            URL url = (URL) json;
-            return path.read(url, configuration);
+        } else {
+            InputStream is = exchange.getIn().getMandatoryBody(InputStream.class);
+            String jsonEncoding = exchange.getIn().getHeader(JsonPathConstants.HEADER_JSON_ENCODING, String.class);
+            if (jsonEncoding != null) {
+                // json encoding specified in header
+                return path.read(is, jsonEncoding, configuration);
+            } else {
+                // no json encoding specified --> assume json encoding is unicode and determine the specific unicode encoding according to RFC-4627
+                // this is a temporary solution, it can be removed as sson as jsonpath offers the encoding detection
+                JsonStream jsonStream = new JsonStream(is);
+                return path.read(jsonStream, jsonStream.getEncoding().name(), configuration);
+            }
         }
-
-        // fallback as input stream
-        InputStream is = exchange.getIn().getMandatoryBody(InputStream.class);
-        return path.read(is);
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonStream.java
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonStream.java b/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonStream.java
new file mode 100644
index 0000000..d0daa5a
--- /dev/null
+++ b/components/camel-jsonpath/src/main/java/org/apache/camel/jsonpath/JsonStream.java
@@ -0,0 +1,292 @@
+/**
+ * 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.jsonpath;
+
+import java.io.CharConversionException;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+
+/**
+ * Special stream for JSON streams. Determines from the first 4 bytes the JSON
+ * encoding according to JSON specification RFC-4627 or newer. In addition BOMs
+ * are taken into account.
+ * <p>
+ * This class is not thread safe.
+ */
+public class JsonStream extends FilterInputStream {
+
+    private static final byte[] BOM_UTF_32BE = new byte[] {0x00, 0x00, (byte) 0xFE, (byte) 0xFF };
+
+    private static final byte[] BOM_UTF_32LE = new byte[] {(byte) 0xFF, (byte) 0xFE, 0x00, 0x00 };
+
+    private static final byte[] BOM_UTF_32_2143 = new byte[] {0x00, 0x00, (byte) 0xFF, (byte) 0xFE };
+
+    private static final byte[] BOM_UTF_32_3412 = new byte[] {(byte) 0xFE, (byte) 0xFF, 0x00, 0x00 };
+
+    private static final byte[] BOM_UTF_16BE = new byte[] {(byte) 0xFE, (byte) 0xFF };
+
+    private static final byte[] BOM_UTF_16LE = new byte[] {(byte) 0xFF, (byte) 0xFE };
+
+    private static final byte[] BOM_UTF_8 = new byte[] {(byte) 0xEF, (byte) 0xBB, (byte) 0xBF };
+
+    private final byte[] fourByteBuffer = new byte[4];
+
+    /* input index of the four byte buffer (BOMs are skipped) */
+    private int inputIndex;
+
+    /* read bytes into the buffer */
+    private int inputEnd;
+
+    private final Charset encoding;
+
+    /**
+     * Constructor. Determines the encoding during the instantiation according
+     * to JSON specification RFC-4627 or newer. In addition BOMs are taken into
+     * account.
+     * 
+     * @param in
+     *            input stream must contain a JSON content
+     * @throws IOException
+     *             if an error occurs during the determination of the encoding
+     * @throws CharConversionException
+     *             if the UCS4 endianess 2143 or 3412 is used
+     * @throws IllegalArgumentException
+     *             if the input stream is <code>null</code>
+     */
+    public JsonStream(InputStream in) throws IOException {
+        super(in);
+        if (in == null) {
+            throw new IllegalArgumentException("input stream is null");
+        }
+
+        inputEnd = inputIndex = 0;
+
+        Charset enc = null;
+
+        if (loadAtLeast(4)) {
+
+            enc = getEncodingFromBOM();
+            if (enc == null) {
+                // no BOM
+                enc = getUTF32EncodingFromNullPattern();
+                if (enc == null) {
+                    enc = getUTF16EncodingFromNullPattern();
+                }
+            }
+        } else if (loadAtLeast(2)) {
+            enc = getUTF16EncodingFromNullPattern();
+        }
+
+        if (enc == null) {
+            // not found; as per specification, this means it must be UTF-8.
+            enc = Charset.forName("UTF-8");
+        }
+        encoding = enc;
+    }
+
+    public Charset getEncoding() {
+        return encoding;
+    }
+
+    private boolean loadAtLeast(int minimum) throws IOException {
+
+        int received = inputEnd - inputIndex;
+        while (received < minimum) {
+            int count = in.read(fourByteBuffer, inputEnd, fourByteBuffer.length - inputEnd);
+            if (count < 1) {
+                return false;
+            }
+            inputEnd += count;
+            received += count;
+        }
+        return true;
+    }
+
+    private Charset getEncodingFromBOM() throws IOException {
+        // 32-bit encoding BOMs       
+        if (Arrays.equals(fourByteBuffer, BOM_UTF_32BE)) {
+            inputIndex = 4;
+            return Charset.forName("UTF-32BE");
+        } else if (Arrays.equals(fourByteBuffer, BOM_UTF_32LE)) {
+            inputIndex = 4;
+            return Charset.forName("UTF-32LE");
+        } else if (Arrays.equals(fourByteBuffer, BOM_UTF_32_2143)) {
+            throw getExceptionUnsupportedUCS4("2143");
+        } else if (Arrays.equals(fourByteBuffer, BOM_UTF_32_3412)) {
+            throw getExceptionUnsupportedUCS4("3412");
+        }
+
+        byte[] firstTwoBytes = Arrays.copyOf(fourByteBuffer, 2);
+        //  16-bit encoding BOMs
+        if (Arrays.equals(firstTwoBytes, BOM_UTF_16BE)) {
+            inputIndex = 2;
+            return Charset.forName("UTF-16BE");
+        }
+        if (Arrays.equals(firstTwoBytes, BOM_UTF_16LE)) {
+            inputIndex = 2;
+            return Charset.forName("UTF-16LE");
+        }
+
+        byte[] firstThreeBytes = Arrays.copyOf(fourByteBuffer, 3);
+        // UTF-8 BOM?
+        if (Arrays.equals(firstThreeBytes, BOM_UTF_8)) {
+            inputIndex = 3;
+            return Charset.forName("UTF-8");
+        }
+        return null;
+    }
+
+    private Charset getUTF32EncodingFromNullPattern() throws IOException {
+        //content without BOM      
+        if (fourByteBuffer[0] == 0 && fourByteBuffer[1] == 0 && fourByteBuffer[2] == 0) {
+            //  00 00 00 xx
+            return Charset.forName("UTF-32BE");
+        } else if (fourByteBuffer[1] == 0 && fourByteBuffer[2] == 0 && fourByteBuffer[3] == 0) {
+            // xx 00 00 00
+            return Charset.forName("UTF-32LE");
+        } else if (fourByteBuffer[0] == 0 && fourByteBuffer[2] == 0 && fourByteBuffer[3] == 0) {
+            // 00 xx 00 00
+            throw getExceptionUnsupportedUCS4("3412");
+        } else if (fourByteBuffer[0] == 0 && fourByteBuffer[1] == 0 && fourByteBuffer[3] == 0) {
+            //00 00 xx 00
+            throw getExceptionUnsupportedUCS4("2143");
+        } else {
+            // Cannot be valid UTF-32 encoded JSON...
+            return null;
+        }
+    }
+
+    private Charset getUTF16EncodingFromNullPattern() {
+        if (fourByteBuffer[0] == 0) {
+            return Charset.forName("UTF-16BE");
+        } else if (fourByteBuffer[1] == 0) {
+            return Charset.forName("UTF-16LE");
+        } else { // not  UTF-16
+            return null;
+        }
+    }
+
+    private CharConversionException getExceptionUnsupportedUCS4(String type) throws IOException {
+        return new CharConversionException("Unsupported UCS-4 endianness (" + type + ") detected");
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (inputIndex < inputEnd) {
+            int result = fourByteBuffer[inputIndex];
+            inputIndex++;
+            return result;
+        }
+        try {
+            return in.read();
+        } catch (java.io.EOFException ex) {
+            return -1;
+        }
+    }
+
+    @Override
+    public int read(byte b[]) throws IOException {
+
+        if (inputIndex < inputEnd) {
+            int minimum = Math.min(b.length, inputEnd - inputIndex);
+            for (int i = 0; i < minimum; i++) {
+                b[i] = fourByteBuffer[inputIndex];
+                inputIndex++;
+            }
+            int rest = b.length - minimum;
+            if (rest == 0) {
+                return minimum;
+            }
+            try {
+                int additionalRead = in.read(b, minimum, rest);
+                if (additionalRead < 0) {
+                    return minimum;
+                } else {
+                    return minimum + additionalRead;
+                }
+            } catch (java.io.EOFException ex) {
+                return minimum;
+            }
+        } else {
+            return read(b, 0, b.length);
+        }
+    }
+
+    @Override
+    public int read(byte b[], int off, int len) throws IOException {
+        if (inputIndex < inputEnd) {
+            int minimum = Math.min(b.length, inputEnd - inputIndex);
+            for (int i = 0; i < minimum; i++) {
+                b[off + i] = fourByteBuffer[inputIndex];
+                inputIndex++;
+            }
+            int rest = b.length - minimum;
+            if (rest == 0) {
+                return minimum;
+            }
+            try {
+                int additionalRead = in.read(b, minimum + off, rest);
+                if (additionalRead < 0) {
+                    return minimum;
+                } else {
+                    return minimum + additionalRead;
+                }
+            } catch (java.io.EOFException ex) {
+                return minimum;
+            }
+        } else {
+
+            try {
+                return in.read(b, off, len);
+            } catch (java.io.EOFException ex) {
+                return -1;
+            }
+        }
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        if (inputIndex < inputEnd) {
+            long minimum = Math.min(n, (long) (inputEnd - inputIndex));
+            for (int i = 0; i < minimum; i++) {
+                inputIndex++;
+            }
+            long rest = n - minimum;
+            if (rest == 0) {
+                return minimum;
+            }
+            long additionalSkipped = in.skip(rest);
+            return additionalSkipped + minimum;
+        } else {
+            return in.skip(n);
+        }
+    }
+
+    @Override
+    public synchronized void reset() throws IOException {
+        throw new IOException("reset not supported");
+    }
+
+    @Override
+    public boolean markSupported() {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonPathCharsetTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonPathCharsetTest.java b/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonPathCharsetTest.java
new file mode 100644
index 0000000..262ac90
--- /dev/null
+++ b/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonPathCharsetTest.java
@@ -0,0 +1,112 @@
+/**
+ * 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.jsonpath;
+
+import java.io.File;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JsonPathCharsetTest extends CamelTestSupport {
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start").transform().jsonpath("$.store.book[*].title").to("mock:authors");
+            }
+        };
+    }
+
+    @Test
+    public void testUTF16BEFile() throws Exception {
+        getMockEndpoint("mock:authors").expectedMessageCount(1);
+
+        sendBody("direct:start", new File("src/test/resources/booksUTF16BE.json"));
+
+        assertMockEndpointsSatisfied();
+
+        check();
+    }
+
+    @Test
+    public void testUTF16LEFile() throws Exception {
+        getMockEndpoint("mock:authors").expectedMessageCount(1);
+
+        sendBody("direct:start", new File("src/test/resources/booksUTF16LE.json"));
+
+        assertMockEndpointsSatisfied();
+
+        check();
+    }
+
+    @Test
+    public void testUTF16BEInputStream() throws Exception {
+        getMockEndpoint("mock:authors").expectedMessageCount(1);
+
+        InputStream input = JsonPathCharsetTest.class.getClassLoader().getResourceAsStream("booksUTF16BE.json");
+        Assert.assertNotNull(input);
+        sendBody("direct:start", input);
+
+        assertMockEndpointsSatisfied();
+
+        check();
+    }
+
+    @Test
+    public void testUTF16BEURL() throws Exception {
+        getMockEndpoint("mock:authors").expectedMessageCount(1);
+
+        URL url = new URL("file:src/test/resources/booksUTF16BE.json");
+        Assert.assertNotNull(url);
+        sendBody("direct:start", url);
+
+        check();
+    }
+
+    @Test
+    public void testISO8859WithJsonHeaderCamelJsonInputEncoding() throws Exception {
+        getMockEndpoint("mock:authors").expectedMessageCount(1);
+
+        URL url = new URL("file:src/test/resources/germanbooks-iso-8859-1.json");
+        Assert.assertNotNull(url);
+        sendBody("direct:start", url, Collections.<String, Object> singletonMap(JsonPathConstants.HEADER_JSON_ENCODING, "ISO-8859-1"));
+
+        check("Joseph und seine Brüder", "Götzendämmerung");
+    }
+
+    private void check() throws InterruptedException {
+        check("Sayings of the Century", "Sword of Honour");
+    }
+
+    private void check(String title1, String title2) throws InterruptedException {
+        assertMockEndpointsSatisfied();
+
+        List<?> authors = getMockEndpoint("mock:authors").getReceivedExchanges().get(0).getIn().getBody(List.class);
+
+        assertEquals(title1, authors.get(0));
+        assertEquals(title2, authors.get(1));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonStreamTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonStreamTest.java b/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonStreamTest.java
new file mode 100644
index 0000000..3412009
--- /dev/null
+++ b/components/camel-jsonpath/src/test/java/org/apache/camel/jsonpath/JsonStreamTest.java
@@ -0,0 +1,130 @@
+/**
+ * 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.jsonpath;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+
+public class JsonStreamTest {
+
+    @Test
+    public void utf8() throws Exception {
+        test("json_stream/jsonUTF8.txt", "UTF-8");
+    }
+
+    @Test
+    public void iSO88591() throws Exception {
+        try {
+            test("json_stream/jsonISO8859-1.txt", "ISO-8859-1");
+            fail("Error exepcted");
+        } catch (AssertionError e) {
+            assertEquals("expected:<ISO-8859-1> but was:<UTF-8>", e.getMessage());
+        }
+    }
+
+    @Test
+    public void utf8WithoutBOM() throws Exception {
+        test("json_stream/jsonUTF8WithoutBOM.txt", "UTF-8");
+    }
+
+    @Test
+    public void utf16BEWithBom() throws Exception {
+        test("json_stream/jsonUCS2BigEndianWithBOM.txt", "UTF-16BE");
+    }
+
+    @Test
+    public void utf16BEWithoutBom() throws Exception {
+        test("json_stream/jsonUCS2BigEndianWithoutBOM.txt", "UTF-16BE");
+    }
+
+    @Test
+    public void utf16LEWithBom() throws Exception {
+        test("json_stream/jsonUCS2LittleEndianWithBom.txt", "UTF-16LE");
+    }
+
+    @Test
+    public void utf16LEWithoutBom() throws Exception {
+        test("json_stream/jsonUCS2LittleEndianWithoutBOM.txt", "UTF-16LE");
+    }
+
+    @Test
+    public void utf32BEWithBOM() throws Exception {
+        test("json_stream/jsonUTF32BEWithBOM.txt", "UTF-32BE");
+    }
+
+    @Test
+    public void utf32BEWithoutBOM() throws Exception {
+        test("json_stream/jsonUTF32BEWithoutBOM.txt", "UTF-32BE");
+    }
+
+    @Test
+    public void utf32LEWithBOM() throws Exception {
+        test("json_stream/jsonUTF32LEWithBOM.txt", "UTF-32LE");
+    }
+
+    @Test
+    public void utf32LEWithoutBOM() throws Exception {
+        test("json_stream/jsonUTF32LEWithoutBOM.txt", "UTF-32LE");
+    }
+
+    @Test
+    public void oneChar() throws Exception {
+        test("json_stream/oneChar", "UTF-8", "1");
+    }
+
+    @Test
+    public void fourChar() throws Exception {
+        test("json_stream/fourChar", "UTF-8", "1234");
+    }
+
+    private void test(String file, String encoding) throws Exception {
+        test(file,
+                encoding,
+                "{ \"a\": \"1\", \"b\": \"2\", \"c\": { \"a\": \"c.a.1\", \"b\": \"c.b.2\" }, \"d\": [\"a\", \"b\", \"c\"], \"e\": [1, 2, 3], \"f\": true, \"g\": null}");
+    }
+
+    private void test(String file, String encoding, String expectedString) throws Exception {
+        InputStream is = JsonStreamTest.class.getClassLoader().getResourceAsStream(file);
+        assertNotNull("File " + file + " not found", is);
+        JsonStream js = new JsonStream(is);
+        Charset actual = js.getEncoding();
+        Charset expected = Charset.forName(encoding);
+        assertEquals(expected, actual);
+
+        byte[] result = readBytes(js);
+        String actualString = new String(result, js.getEncoding());
+        assertEquals(expectedString, actualString);
+    }
+
+    byte[] readBytes(JsonStream js) throws IOException {
+        // read all
+        byte[] buffer = new byte[2048];
+        int len = js.read(buffer);
+        js.close();
+        byte[] result = Arrays.copyOf(buffer, len);
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/booksUTF16BE.json
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/booksUTF16BE.json b/components/camel-jsonpath/src/test/resources/booksUTF16BE.json
new file mode 100644
index 0000000..cd4c3ed
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/booksUTF16BE.json differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/booksUTF16LE.json
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/booksUTF16LE.json b/components/camel-jsonpath/src/test/resources/booksUTF16LE.json
new file mode 100644
index 0000000..7e066b1
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/booksUTF16LE.json differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/fourChar
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/fourChar b/components/camel-jsonpath/src/test/resources/json_stream/fourChar
new file mode 100644
index 0000000..274c005
--- /dev/null
+++ b/components/camel-jsonpath/src/test/resources/json_stream/fourChar
@@ -0,0 +1 @@
+1234
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonISO8859-1.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonISO8859-1.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonISO8859-1.txt
new file mode 100644
index 0000000..cc31465
--- /dev/null
+++ b/components/camel-jsonpath/src/test/resources/json_stream/jsonISO8859-1.txt
@@ -0,0 +1 @@
+{ "a": "1", "b": "2", "c": { "a": "c.a.1", "b": "c.b.2" }, "d": ["a", "b", "c"], "e": [1, 2, 3], "f": true, "g": null}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithBOM.txt
new file mode 100644
index 0000000..ca78946
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithBOM.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithoutBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithoutBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithoutBOM.txt
new file mode 100644
index 0000000..202e379
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2BigEndianWithoutBOM.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithBom.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithBom.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithBom.txt
new file mode 100644
index 0000000..35efa85
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithBom.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithoutBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithoutBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithoutBOM.txt
new file mode 100644
index 0000000..9a930fb
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUCS2LittleEndianWithoutBOM.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithBOM.txt
new file mode 100644
index 0000000..3ac14e6
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithBOM.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithoutBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithoutBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithoutBOM.txt
new file mode 100644
index 0000000..1bb2cd6
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32BEWithoutBOM.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithBOM.txt
new file mode 100644
index 0000000..796bc83
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithBOM.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithoutBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithoutBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithoutBOM.txt
new file mode 100644
index 0000000..64d9d9f
Binary files /dev/null and b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF32LEWithoutBOM.txt differ

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8.txt
new file mode 100644
index 0000000..aa001ae
--- /dev/null
+++ b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8.txt
@@ -0,0 +1 @@
+{ "a": "1", "b": "2", "c": { "a": "c.a.1", "b": "c.b.2" }, "d": ["a", "b", "c"], "e": [1, 2, 3], "f": true, "g": null}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8WithoutBOM.txt
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8WithoutBOM.txt b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8WithoutBOM.txt
new file mode 100644
index 0000000..cc31465
--- /dev/null
+++ b/components/camel-jsonpath/src/test/resources/json_stream/jsonUTF8WithoutBOM.txt
@@ -0,0 +1 @@
+{ "a": "1", "b": "2", "c": { "a": "c.a.1", "b": "c.b.2" }, "d": ["a", "b", "c"], "e": [1, 2, 3], "f": true, "g": null}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/604d2d8c/components/camel-jsonpath/src/test/resources/json_stream/oneChar
----------------------------------------------------------------------
diff --git a/components/camel-jsonpath/src/test/resources/json_stream/oneChar b/components/camel-jsonpath/src/test/resources/json_stream/oneChar
new file mode 100644
index 0000000..56a6051
--- /dev/null
+++ b/components/camel-jsonpath/src/test/resources/json_stream/oneChar
@@ -0,0 +1 @@
+1
\ No newline at end of file