You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by ni...@apache.org on 2009/03/12 04:25:01 UTC

svn commit: r752756 - in /camel/branches/camel-1.x: ./ camel-core/src/main/java/org/apache/camel/converter/stream/ camel-core/src/main/java/org/apache/camel/util/ camel-core/src/test/java/org/apache/camel/converter/stream/

Author: ningjiang
Date: Thu Mar 12 03:24:59 2009
New Revision: 752756

URL: http://svn.apache.org/viewvc?rev=752756&view=rev
Log:
Merged revisions 749936 via svnmerge from 
https://svn.apache.org/repos/asf/camel/trunk

........
  r749936 | ningjiang | 2009-03-04 15:34:17 +0800 (Wed, 04 Mar 2009) | 1 line
  
  CAMEL-1413 Cached the big message into the file instead of using memory directly
........

Added:
    camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java
      - copied, changed from r749936, camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java
    camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java
      - copied, changed from r749936, camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java
    camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java
      - copied, changed from r749936, camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java
    camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/FileUtil.java   (with props)
    camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java
      - copied, changed from r749936, camel/trunk/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java
Modified:
    camel/branches/camel-1.x/   (props changed)
    camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/StreamCacheConverter.java
    camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/IOHelper.java
    camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/StreamCacheConverterTest.java

Propchange: camel/branches/camel-1.x/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Mar 12 03:24:59 2009
@@ -1 +1 @@
-/camel/trunk:736980,739733,739904,740251,740295,740306,740596,740663,741848,742231,742705,742739,742854,742856,742898,742906,743613,743762,743773,743920,743959-743960,744123,745105,745367,745541,745751,745826,745978,746269,746872,746895,746962,747258,747678-747704,748392,748436,748821,749563-749564,749574,749628-749629,749956,750017,750334,750761,750796,752068,752117
+/camel/trunk:736980,739733,739904,740251,740295,740306,740596,740663,741848,742231,742705,742739,742854,742856,742898,742906,743613,743762,743773,743920,743959-743960,744123,745105,745367,745541,745751,745826,745978,746269,746872,746895,746962,747258,747678-747704,748392,748436,748821,749563-749564,749574,749628-749629,749936,749956,750017,750334,750761,750796,752068,752117

Propchange: camel/branches/camel-1.x/
------------------------------------------------------------------------------
Binary property 'svnmerge-integrated' - no diff available.

Copied: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java (from r749936, camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java)
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java?p2=camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java&r1=749936&r2=752756&rev=752756&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java (original)
+++ camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/CachedOutputStream.java Thu Mar 12 03:24:59 2009
@@ -29,7 +29,6 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.camel.StreamCache;
 import org.apache.camel.converter.IOConverter;
 import org.apache.camel.util.FileUtil;
 import org.apache.camel.util.IOHelper;

Copied: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java (from r749936, camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java)
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java?p2=camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java&r1=749936&r2=752756&rev=752756&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java (original)
+++ camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/FileInputStreamCache.java Thu Mar 12 03:24:59 2009
@@ -23,7 +23,6 @@
 import java.io.InputStream;
 
 import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.StreamCache;
 
 public class FileInputStreamCache extends InputStream implements StreamCache {
     private FileInputStream inputStream;

Copied: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java (from r749936, camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java)
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java?p2=camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java&r1=749936&r2=752756&rev=752756&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java (original)
+++ camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/InputStreamCache.java Thu Mar 12 03:24:59 2009
@@ -19,9 +19,6 @@
 
 import java.io.ByteArrayInputStream;
 
-import org.apache.camel.StreamCache;
-
-
 public class InputStreamCache extends ByteArrayInputStream implements StreamCache {
 
     public InputStreamCache(byte[] data) {

Modified: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/StreamCacheConverter.java
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/StreamCacheConverter.java?rev=752756&r1=752755&r2=752756&view=diff
==============================================================================
--- camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/StreamCacheConverter.java (original)
+++ camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/converter/stream/StreamCacheConverter.java Thu Mar 12 03:24:59 2009
@@ -16,7 +16,6 @@
  */
 package org.apache.camel.converter.stream;
 
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
@@ -27,10 +26,12 @@
 import javax.xml.transform.stream.StreamSource;
 
 import org.apache.camel.Converter;
+import org.apache.camel.Exchange;
 import org.apache.camel.converter.IOConverter;
 import org.apache.camel.converter.jaxp.BytesSource;
 import org.apache.camel.converter.jaxp.StringSource;
 import org.apache.camel.converter.jaxp.XmlConverter;
+import org.apache.camel.util.IOHelper;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -67,8 +68,10 @@
     }
 
     @Converter
-    public StreamCache convertToStreamCache(InputStream stream) throws IOException {
-        return new InputStreamCache(IOConverter.toBytes(stream));
+    public StreamCache convertToStreamCache(InputStream stream, Exchange exchange) throws IOException {
+        CachedOutputStream cos = new CachedOutputStream();
+        IOHelper.copyAndCloseInput(stream, cos);       
+        return cos.getStreamCache();
     }
 
     @Converter
@@ -100,13 +103,15 @@
      * {@link StreamCache} implementation for Cache the StreamSource {@link StreamSource}s
      */
     private class StreamSourceCache extends StreamSource implements StreamCache {
-        InputStreamCache inputStreamCache;
+        StreamCache streamCache;
         ReaderCache readCache;
         
         public StreamSourceCache(StreamSource source) throws IOException {
             if (source.getInputStream() != null) {
-                inputStreamCache = new InputStreamCache(IOConverter.toBytes(source.getInputStream()));
-                setInputStream(inputStreamCache);
+                CachedOutputStream cos = new CachedOutputStream();
+                IOHelper.copyAndCloseInput(source.getInputStream(), cos);
+                streamCache = cos.getStreamCache();
+                setInputStream((InputStream)streamCache);
                 setSystemId(source.getSystemId());
             }
             if (source.getReader() != null) {
@@ -115,8 +120,8 @@
             }
         }
         public void reset() {
-            if (inputStreamCache != null) {
-                inputStreamCache.reset();
+            if (streamCache != null) {
+                streamCache.reset();
             }
             if (readCache != null) {
                 readCache.reset();
@@ -124,15 +129,7 @@
         }
         
     }
-
-    private class InputStreamCache extends ByteArrayInputStream implements StreamCache {
-
-        public InputStreamCache(byte[] data) {
-            super(data);
-        }
-
-    }
-
+      
     private class ReaderCache extends StringReader implements StreamCache {
 
         public ReaderCache(String s) {
@@ -152,6 +149,8 @@
         }
 
     }
+    
+    
 
 
 }

Added: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/FileUtil.java
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/FileUtil.java?rev=752756&view=auto
==============================================================================
--- camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/FileUtil.java (added)
+++ camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/FileUtil.java Thu Mar 12 03:24:59 2009
@@ -0,0 +1,388 @@
+/**
+ * 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.util;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Stack;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * File utilities
+ */
+public final class FileUtil {
+    
+    private static final int RETRY_SLEEP_MILLIS = 10;
+    private static File defaultTempDir;
+
+    private FileUtil() {
+    }
+
+    /**
+     * Normalizes the path to cater for Windows and other platforms
+     */
+    public static String normalizePath(String path) {
+        // special handling for Windows where we need to convert / to \\
+        if (path != null && System.getProperty("os.name").startsWith("Windows") && path.indexOf("/") >= 0) {
+            return path.replace('/', '\\');
+        }
+        return path;
+    }
+    
+    private static synchronized File getDefaultTempDir() {
+        if (defaultTempDir != null
+            && defaultTempDir.exists()) {
+            return defaultTempDir;
+        }
+        
+        String s = null;
+        try {
+            s = System.getProperty(FileUtil.class.getName() + ".TempDirectory");
+        } catch (SecurityException e) {
+            //Ignorable, we'll use the default
+        }
+        if (s == null) {
+            int x = (int)(Math.random() * 1000000);
+            s = System.getProperty("java.io.tmpdir");
+            File checkExists = new File(s);
+            if (!checkExists.exists()) {
+                throw new RuntimeException("The directory " 
+                                       + checkExists.getAbsolutePath() 
+                                       + " does not exist, please set java.io.tempdir"
+                                       + " to an existing directory");
+            }
+            File f = new File(s, "camel-tmp-" + x);
+            while (!f.mkdir()) {
+                x = (int)(Math.random() * 1000000);
+                f = new File(s, "camel-tmp-" + x);
+            }
+            defaultTempDir = f;
+            Thread hook = new Thread() {
+                @Override
+                public void run() {
+                    removeDir(defaultTempDir);
+                }
+            };
+            Runtime.getRuntime().addShutdownHook(hook);            
+        } else {
+            //assume someone outside of us will manage the directory
+            File f = new File(s);
+            f.mkdirs();
+            defaultTempDir = f;
+        }
+        return defaultTempDir;
+    }
+
+    public static void mkDir(File dir) {
+        if (dir == null) {
+            throw new RuntimeException("dir attribute is required");
+        }
+
+        if (dir.isFile()) {
+            throw new RuntimeException("Unable to create directory as a file "
+                                    + "already exists with that name: " + dir.getAbsolutePath());
+        }
+
+        if (!dir.exists()) {
+            boolean result = doMkDirs(dir);
+            if (!result) {
+                String msg = "Directory " + dir.getAbsolutePath()
+                             + " creation was not successful for an unknown reason";
+                throw new RuntimeException(msg);
+            }
+        }
+    }
+
+    /**
+     * Attempt to fix possible race condition when creating directories on
+     * WinXP, also Windows2000. If the mkdirs does not work, wait a little and
+     * try again.
+     */
+    private static boolean doMkDirs(File f) {
+        if (!f.mkdirs()) {
+            try {
+                Thread.sleep(RETRY_SLEEP_MILLIS);
+                return f.mkdirs();
+            } catch (InterruptedException ex) {
+                return f.mkdirs();
+            }
+        }
+        return true;
+    }
+
+    public static void removeDir(File d) {
+        String[] list = d.list();
+        if (list == null) {
+            list = new String[0];
+        }
+        for (int i = 0; i < list.length; i++) {
+            String s = list[i];
+            File f = new File(d, s);
+            if (f.isDirectory()) {
+                removeDir(f);
+            } else {
+                delete(f);
+            }
+        }
+        delete(d);
+    }
+
+    public static void delete(File f) {
+        if (!f.delete()) {
+            if (isWindows()) {
+                System.gc();
+            }
+            try {
+                Thread.sleep(RETRY_SLEEP_MILLIS);
+            } catch (InterruptedException ex) {
+                // Ignore Exception
+            }
+            if (!f.delete()) {
+                f.deleteOnExit();
+            }
+        }
+    }
+
+    private static boolean isWindows() {
+        String osName = System.getProperty("os.name").toLowerCase(Locale.US);
+        return osName.indexOf("windows") > -1;
+    }
+
+    public static File createTempFile(String prefix, String suffix) throws IOException {
+        return createTempFile(prefix, suffix, null, false);
+    }
+    
+    public static File createTempFile(String prefix, String suffix, File parentDir,
+                               boolean deleteOnExit) throws IOException {
+        File result = null;
+        File parent = (parentDir == null)
+            ? getDefaultTempDir()
+            : parentDir;
+            
+        if (suffix == null) {
+            suffix = ".tmp";
+        }
+        if (prefix == null) {
+            prefix = "camel";
+        } else if (prefix.length() < 3) {
+            prefix = prefix + "camel";
+        }
+        result = File.createTempFile(prefix, suffix, parent);
+
+        //if parentDir is null, we're in our default dir
+        //which will get completely wiped on exit from our exit
+        //hook.  No need to set deleteOnExit() which leaks memory.
+        if (deleteOnExit && parentDir != null) {
+            result.deleteOnExit();
+        }
+        return result;
+    }
+    
+    public static String getStringFromFile(File location) {
+        InputStream is = null;
+        String result = null;
+
+        try {
+            is = new FileInputStream(location);
+            result = normalizeCRLF(is);
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (Exception e) {
+                    //do nothing
+                }
+            }
+        }
+
+        return result;
+    }
+
+    public static String normalizeCRLF(InputStream instream) {
+        BufferedReader in = new BufferedReader(new InputStreamReader(instream));
+        StringBuffer result = new StringBuffer();
+        String line = null;
+
+        try {
+            line = in.readLine();
+            while (line != null) {
+                String[] tok = line.split("\\s");
+
+                for (int x = 0; x < tok.length; x++) {
+                    String token = tok[x];
+                    result.append("  " + token);
+                }
+                line = in.readLine();
+            }
+        } catch (Exception ex) {
+            ex.printStackTrace();
+        }
+
+        String rtn = result.toString();
+
+        rtn = ignoreTokens(rtn, "<!--", "-->");
+        rtn = ignoreTokens(rtn, "/*", "*/");
+        return rtn;
+    }
+    
+    private static String ignoreTokens(final String contents, 
+                                       final String startToken, final String endToken) {
+        String rtn = contents;
+        int headerIndexStart = rtn.indexOf(startToken);
+        int headerIndexEnd = rtn.indexOf(endToken);
+        if (headerIndexStart != -1 && headerIndexEnd != -1 && headerIndexStart < headerIndexEnd) {
+            rtn = rtn.substring(0, headerIndexStart - 1)
+                + rtn.substring(headerIndexEnd + endToken.length() + 1);
+        }
+        return rtn;
+    }
+
+    public static List<File> getFiles(File dir, final String pattern) {
+        return getFiles(dir, pattern, null);
+    }
+    public static List<File> getFilesRecurse(File dir, final String pattern) {
+        return getFilesRecurse(dir, pattern, null);
+    }
+
+    public static List<File> getFiles(File dir, final String pattern, File exclude) {
+        return getFilesRecurse(dir, Pattern.compile(pattern), exclude, false, new ArrayList<File>());
+    }
+    public static List<File> getFilesRecurse(File dir, final String pattern, File exclude) {
+        return getFilesRecurse(dir, Pattern.compile(pattern), exclude, true, new ArrayList<File>());    
+    }
+    private static List<File> getFilesRecurse(File dir, 
+                                              Pattern pattern,
+                                              File exclude, boolean rec,
+                                              List<File> fileList) {
+        for (File file : dir.listFiles()) {
+            if (file.equals(exclude)) {
+                continue;
+            }
+            if (file.isDirectory() && rec) {
+                getFilesRecurse(file, pattern, exclude, rec, fileList);
+            } else {
+                Matcher m = pattern.matcher(file.getName());
+                if (m.matches()) {
+                    fileList.add(file);                                
+                }
+            }
+        }
+        return fileList;
+    }
+
+    public static List<String> readLines(File file) throws Exception {
+        if (!file.exists()) {
+            return new ArrayList<String>();
+        }
+        BufferedReader reader = new BufferedReader(new FileReader(file));
+        List<String> results = new ArrayList<String>();
+        String line = reader.readLine();
+        while (line != null) {
+            results.add(line);
+            line = reader.readLine();
+        }
+        return results;
+    }
+
+    /**
+     * Strip any leading separators
+     */
+    public static String stripLeadingSeparator(String name) {
+        if (name == null) {
+            return null;
+        }
+        while (name.startsWith("/") || name.startsWith(File.separator)) {
+            name = name.substring(1);
+        }
+        return name;
+    }
+
+    /**
+     * Strip any trailing separators
+     */
+    public static String stripTrailingSeparator(String name) {
+        if (name == null) {
+            return null;
+        }
+        while (name.endsWith("/") || name.endsWith(File.separator)) {
+            name = name.substring(0, name.length() - 1);
+        }
+        return name;
+    }
+
+    /**
+     * Strips any leading paths
+     */
+    public static String stripPath(String name) {
+        if (name == null) {
+            return null;
+        }
+        int pos = name.lastIndexOf("/");
+        if (pos == -1) {
+            pos = name.lastIndexOf(File.separator);
+        }
+        if (pos != -1) {
+            return name.substring(pos + 1);
+        }
+        return name;
+    }
+
+    /**
+     * Compacts a path by stacking it and reducing <tt>..</tt>
+     */
+    public static String compactPath(String path) {
+        // only normalize path if it contains .. as we want to avoid: path/../sub/../sub2 as this can leads to trouble
+        if (path.indexOf("..") == -1) {
+            return path;
+        }
+
+        Stack<String> stack = new Stack<String>();
+        String[] parts = path.split(File.separator);
+        for (String part : parts) {
+            if (part.equals("..") && !stack.isEmpty()) {
+                // only pop if there is a previous path
+                stack.pop();
+            } else {
+                stack.push(part);
+            }
+        }
+
+        // build path based on stack
+        StringBuilder sb = new StringBuilder();
+        for (Iterator it = stack.iterator(); it.hasNext();) {
+            sb.append(it.next());
+            if (it.hasNext()) {
+                sb.append(File.separator);
+            }
+        }
+
+        return sb.toString();
+    }
+
+}

Propchange: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/FileUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/FileUtil.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/IOHelper.java
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/IOHelper.java?rev=752756&r1=752755&r2=752756&view=diff
==============================================================================
--- camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/IOHelper.java (original)
+++ camel/branches/camel-1.x/camel-core/src/main/java/org/apache/camel/util/IOHelper.java Thu Mar 12 03:24:59 2009
@@ -17,6 +17,10 @@
 package org.apache.camel.util;
 
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
 
 /**
  * IO helper class.
@@ -24,10 +28,46 @@
  * @version $Revision$
  */
 public final class IOHelper {
+    
+    private static final int DEFAULT_BUFFER_SIZE = 1024 * 4;
+    private static final Charset UTF8_CHARSET = Charset.forName("utf-8");
 
     private IOHelper() {
         //Utility Class
     }
+    
+    /**
+     * Use this function instead of new String(byte[]) to avoid surprises from non-standard default encodings.
+     * @param bytes
+     * @return
+     */
+    public static String newStringFromBytes(byte[] bytes) {
+        try {
+            return new String(bytes, UTF8_CHARSET.name());
+        } catch (UnsupportedEncodingException e) {
+            throw 
+                new RuntimeException("Impossible failure: Charset.forName(\"utf-8\") returns invalid name.");
+
+        }
+    }
+
+    /**
+     * Use this function instead of new String(byte[], int, int) 
+     * to avoid surprises from non-standard default encodings.
+     * @param bytes
+     * @param start
+     * @param length
+     * @return
+     */
+    public static String newStringFromBytes(byte[] bytes, int start, int length) {
+        try {
+            return new String(bytes, start, length, UTF8_CHARSET.name());
+        } catch (UnsupportedEncodingException e) {
+            throw 
+                new RuntimeException("Impossible failure: Charset.forName(\"utf-8\") returns invalid name.");
+
+        }
+    }
 
     /**
      * A factory method which creates an {@link IOException} from the given
@@ -46,4 +86,40 @@
         answer.initCause(cause);
         return answer;
     }
+
+    public static int copy(InputStream input, OutputStream output) throws IOException {
+        return copy(input, output, DEFAULT_BUFFER_SIZE);
+    }
+    
+    public static int copy(final InputStream input, final OutputStream output, int bufferSize)
+        throws IOException {
+        int avail = input.available();
+        if (avail > 262144) {
+            avail = 262144;
+        }
+        if (avail > bufferSize) {
+            bufferSize = avail;
+        }
+        final byte[] buffer = new byte[bufferSize];
+        int n = 0;
+        n = input.read(buffer);
+        int total = 0;
+        while (-1 != n) {
+            output.write(buffer, 0, n);
+            total += n;
+            n = input.read(buffer);
+        }
+        output.flush();
+        return total;
+    }
+    
+    public static void copyAndCloseInput(InputStream input, OutputStream output) throws IOException {
+        copy(input, output);
+        input.close();
+    }
+    
+    public static void copyAndCloseInput(InputStream input, OutputStream output, int bufferSize) throws IOException {
+        copy(input, output, bufferSize);
+    }
+
 }

Copied: camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java (from r749936, camel/trunk/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java)
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java?p2=camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java&r1=749936&r2=752756&rev=752756&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java (original)
+++ camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/CachedOutputStreamTest.java Thu Mar 12 03:24:59 2009
@@ -23,7 +23,7 @@
 import java.io.InputStreamReader;
 
 import junit.framework.TestCase;
-import org.apache.camel.StreamCache;
+
 import org.apache.camel.converter.IOConverter;
 import org.apache.camel.util.CollectionStringBuffer;
 

Modified: camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/StreamCacheConverterTest.java
URL: http://svn.apache.org/viewvc/camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/StreamCacheConverterTest.java?rev=752756&r1=752755&r2=752756&view=diff
==============================================================================
--- camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/StreamCacheConverterTest.java (original)
+++ camel/branches/camel-1.x/camel-core/src/test/java/org/apache/camel/converter/stream/StreamCacheConverterTest.java Thu Mar 12 03:24:59 2009
@@ -24,23 +24,26 @@
 import javax.xml.transform.TransformerException;
 import javax.xml.transform.stream.StreamSource;
 
-import junit.framework.TestCase;
-
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
 import org.apache.camel.converter.IOConverter;
 import org.apache.camel.converter.jaxp.XmlConverter;
+import org.apache.camel.impl.DefaultExchange;
 
 /**
  * Test cases for {@link StreamCacheConverter}
  */
-public class StreamCacheConverterTest extends TestCase {
+public class StreamCacheConverterTest extends ContextTestSupport {
     
     private static final String TEST_FILE = "org/apache/camel/converter/stream/test.xml";
     private StreamCacheConverter converter;
+    private Exchange exchange;
     
     @Override
     protected void setUp() throws Exception {
         super.setUp();
         this.converter = new StreamCacheConverter();
+        this.exchange = new DefaultExchange(context);
     }
 
     public void testConvertToStreamCacheStreamSource() throws IOException, FileNotFoundException, TransformerException {
@@ -55,7 +58,7 @@
 
     public void testConvertToStreamCacheInputStream() throws IOException {
         InputStream is = getTestFileStream();
-        InputStream cache = (InputStream)converter.convertToStreamCache(is);
+        InputStream cache = (InputStream)converter.convertToStreamCache(is, exchange);
         //assert re-readability of the cached InputStream
         assertNotNull(IOConverter.toString(cache));
         assertNotNull(IOConverter.toString(cache));