You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cxf.apache.org by tl...@apache.org on 2006/10/25 14:36:45 UTC

svn commit: r467624 - in /incubator/cxf/trunk: api/src/main/java/org/apache/cxf/phase/ rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/ rt/bindings/soap/src/test/java/org/apache/cxf/binding/soap/attachments/ rt/core/ rt/core/src/...

Author: tli
Date: Wed Oct 25 05:36:43 2006
New Revision: 467624

URL: http://svn.apache.org/viewvc?view=rev&rev=467624
Log:
refactor the AttachmentSerializer/Deserializer to rt/core, fix the Client-outbound fault catch/processing, add Client-outbound fault type test

Added:
    incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentDeserializer.java   (with props)
    incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentSerializer.java   (with props)
Modified:
    incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java
    incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/AttachmentOutInterceptor.java
    incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/MultipartMessageInterceptor.java
    incubator/cxf/trunk/rt/bindings/soap/src/test/java/org/apache/cxf/binding/soap/attachments/AttachmentTest.java
    incubator/cxf/trunk/rt/core/pom.xml
    incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/interceptor/AbstractFaultChainIntiatorObserver.java
    incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/attachment/JAXBAttachmentUnmarshaller.java
    incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/io/MessageDataReader.java
    incubator/cxf/trunk/rt/frontend/jaxws/src/test/java/org/apache/cxf/jaxws/JaxWsClientTest.java
    incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/mtom/TestMultipartMessageInterceptor.java
    incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/type_test/AbstractTypeTestClient.java
    incubator/cxf/trunk/testutils/pom.xml

Modified: incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java (original)
+++ incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java Wed Oct 25 05:36:43 2006
@@ -154,6 +154,13 @@
                 
                 if (faultObserver != null) {
                     faultObserver.onMessage(message);
+                } else {
+                    // Client out-bound message, directly throw exception back to client
+                    if (message.getExchange() != null 
+                            && message == message.getExchange().getOutMessage() 
+                            && message.containsKey(Message.REQUESTOR_ROLE)) {
+                        throw new RuntimeException(ex);
+                    }
                 }
                 state = State.ABORTED;
             } 

Modified: incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/AttachmentOutInterceptor.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/AttachmentOutInterceptor.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/AttachmentOutInterceptor.java (original)
+++ incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/AttachmentOutInterceptor.java Wed Oct 25 05:36:43 2006
@@ -24,13 +24,13 @@
 import java.util.Collection;
 import java.util.ResourceBundle;
 
+import org.apache.cxf.binding.attachment.AttachmentSerializer;
 import org.apache.cxf.binding.soap.SoapFault;
 import org.apache.cxf.binding.soap.SoapMessage;
 import org.apache.cxf.common.i18n.BundleUtils;
 import org.apache.cxf.common.i18n.Message;
 import org.apache.cxf.interceptor.Fault;
 import org.apache.cxf.io.AbstractCachedOutputStream;
-import org.apache.cxf.jaxb.attachment.AttachmentSerializer;
 import org.apache.cxf.message.Attachment;
 import org.apache.cxf.phase.Phase;
 

Modified: incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/MultipartMessageInterceptor.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/MultipartMessageInterceptor.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/MultipartMessageInterceptor.java (original)
+++ incubator/cxf/trunk/rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/MultipartMessageInterceptor.java Wed Oct 25 05:36:43 2006
@@ -19,7 +19,7 @@
 
 package org.apache.cxf.binding.soap.interceptor;
 
-import org.apache.cxf.jaxb.attachment.AttachmentDeserializer;
+import org.apache.cxf.binding.attachment.AttachmentDeserializer;
 import org.apache.cxf.message.Message;
 import org.apache.cxf.phase.AbstractPhaseInterceptor;
 import org.apache.cxf.phase.Phase;

Modified: incubator/cxf/trunk/rt/bindings/soap/src/test/java/org/apache/cxf/binding/soap/attachments/AttachmentTest.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/bindings/soap/src/test/java/org/apache/cxf/binding/soap/attachments/AttachmentTest.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/bindings/soap/src/test/java/org/apache/cxf/binding/soap/attachments/AttachmentTest.java (original)
+++ incubator/cxf/trunk/rt/bindings/soap/src/test/java/org/apache/cxf/binding/soap/attachments/AttachmentTest.java Wed Oct 25 05:36:43 2006
@@ -46,6 +46,8 @@
 import javax.xml.stream.XMLStreamReader;
 import javax.xml.stream.XMLStreamWriter;
 
+import org.apache.cxf.binding.attachment.AttachmentDeserializer;
+import org.apache.cxf.binding.attachment.AttachmentSerializer;
 import org.apache.cxf.binding.attachment.CachedOutputStream;
 import org.apache.cxf.binding.soap.Soap11;
 import org.apache.cxf.binding.soap.Soap12;
@@ -54,8 +56,6 @@
 import org.apache.cxf.binding.soap.TestUtil;
 import org.apache.cxf.binding.soap.interceptor.MultipartMessageInterceptor;
 import org.apache.cxf.bindings.soap.attachments.types.DetailType;
-import org.apache.cxf.jaxb.attachment.AttachmentDeserializer;
-import org.apache.cxf.jaxb.attachment.AttachmentSerializer;
 import org.apache.cxf.jaxb.attachment.JAXBAttachmentMarshaller;
 import org.apache.cxf.jaxb.attachment.JAXBAttachmentUnmarshaller;
 import org.apache.cxf.message.Attachment;

Modified: incubator/cxf/trunk/rt/core/pom.xml
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/core/pom.xml?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/core/pom.xml (original)
+++ incubator/cxf/trunk/rt/core/pom.xml Wed Oct 25 05:36:43 2006
@@ -54,6 +54,11 @@
         </dependency>
 
         <dependency>
+            <groupId>javax.mail</groupId>
+            <artifactId>mail</artifactId>
+        </dependency>
+        
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>provided</scope>

Added: incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentDeserializer.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentDeserializer.java?view=auto&rev=467624
==============================================================================
--- incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentDeserializer.java (added)
+++ incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentDeserializer.java Wed Oct 25 05:36:43 2006
@@ -0,0 +1,339 @@
+/**
+ * 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.cxf.binding.attachment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PushbackInputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Map;
+
+import javax.activation.DataHandler;
+import javax.activation.DataSource;
+import javax.mail.Header;
+import javax.mail.MessagingException;
+import javax.mail.internet.InternetHeaders;
+
+import org.apache.cxf.io.AbstractCachedOutputStream;
+import org.apache.cxf.message.Attachment;
+import org.apache.cxf.message.Message;
+
+public class AttachmentDeserializer {
+
+    public static final String ATTACHMENT_DIRECTORY = "attachment-directory";
+
+    public static final String ATTACHMENT_MEMORY_THRESHOLD = "attachment-memory-threshold";
+
+    public static final int THRESHHOLD = 1024 * 100;
+
+    private PushbackInputStream stream;
+
+    private String boundary;
+
+    private String contentType;
+
+    private List<CachedOutputStream> cache;
+
+    private Message message;
+
+    public AttachmentDeserializer(Message messageParam) {
+        message = messageParam;
+    }
+
+    public boolean preprocessMessage() {
+        InputStream input;
+        Map httpHeaders;
+        // processing message if its multi-part/form-related
+        try {
+            httpHeaders = (Map) message.get(Message.PROTOCOL_HEADERS);
+            if (httpHeaders == null) {
+                return false;
+            } else {
+                List ctList = (List) httpHeaders.get("Content-Type");
+                if (ctList != null) {
+                    for (int x = 0; x < ctList.size(); x++) {
+                        if (x == 0) {
+                            contentType = (String) ctList.get(x);
+                        } else {
+                            contentType += "; " + (String) ctList.get(x);
+                        }
+                    }
+                }
+                if (contentType == null) { 
+                    return false;
+                }
+                input = message.getContent(InputStream.class);
+                if (input == null) {
+                    return false;
+                }
+            }
+            //printStream(input);
+            if (contentType.toLowerCase().indexOf("multipart/related") != -1) {
+                cache = new ArrayList<CachedOutputStream>();
+                int i = contentType.indexOf("boundary=\"");
+                int end;
+                int len;
+                if (i == -1) {
+                    i = contentType.indexOf("boundary=");
+                    end = contentType.indexOf(";", i + 9);
+                    if (end == -1) {
+                        end = contentType.length();
+                    }
+                    len = 9;
+                } else {
+                    end = contentType.indexOf("\"", i + 10);
+                    len = 10;
+                }
+                if (i == -1 || end == -1) {
+                    throw new IOException("Invalid content type: missing boundary! " + contentType);
+                }
+                boundary = "--" + contentType.substring(i + len, end);
+                stream = new PushbackInputStream(input, boundary.length());
+                if (!readTillFirstBoundary(stream, boundary.getBytes())) {
+                    throw new IOException("Couldn't find MIME boundary: " + boundary);
+                }
+                processSoapBody();
+                return true;
+            }
+        } catch (IOException ioe) {
+            message.setContent(Exception.class, ioe);
+        } catch (MessagingException me) {
+            message.setContent(Exception.class, me);
+        }
+        return false;
+    }
+
+    /**
+     * release the resource
+     */
+    public void dispose() {
+        if (cache != null) {
+            for (CachedOutputStream cos : cache) {
+                cos.dispose();
+            }
+        }
+    }
+
+    public void process() throws MessagingException, IOException {
+        processSoapBody();
+        processAttachments();
+    }
+
+    /**
+     * construct the primary soap body part and attachments
+     */
+    public void processSoapBody() throws MessagingException, IOException {
+
+        Attachment soapMimePart = readMimePart();
+        message.setContent(Attachment.class, soapMimePart);        
+        InputStream in = soapMimePart.getDataHandler().getInputStream();
+        message.setContent(InputStream.class, in);
+    }
+
+    public void processAttachments() throws MessagingException, IOException {
+        Collection<Attachment> attachments = message.getAttachments();
+        Attachment att = readMimePart();
+        while (att != null && att.getId() != null) {
+            attachments.add(att);
+            att = readMimePart();
+        }
+    }
+
+    public Attachment getAttachment(String cid) throws MessagingException, IOException {
+        Collection<Attachment> attachments = message.getAttachments();
+        for (Attachment att : attachments) {
+            if (att.getId().equals(cid)) {
+                return att;
+            }
+        }
+        Attachment att = readMimePart();
+        while (att != null && att.getId() != null) {
+            attachments.add(att);
+            String convertId = cid.substring(0, 4).equals("cid:") ? cid.substring(4) : cid;
+            if (att.getId().equals(convertId)) {
+                return att;
+            }
+            att = readMimePart();
+        }
+        return null;
+    }
+
+    /**
+     * Move the read pointer to the begining of the first part read till the end
+     * of first boundary
+     * 
+     * @param pushbackInStream
+     * @param boundary
+     * @throws MessagingException
+     */
+    private static boolean readTillFirstBoundary(PushbackInputStream pbs, byte[] bp) throws IOException {
+
+        // work around a bug in PushBackInputStream where the buffer isn't
+        // initialized
+        // and available always returns 0.
+        int value = pbs.read();
+        pbs.unread(value);
+        while (value != -1) {
+            value = pbs.read();
+            if ((byte) value == bp[0]) {
+                int boundaryIndex = 0;
+                while (value != -1 && (boundaryIndex < bp.length) && ((byte) value == bp[boundaryIndex])) {
+
+                    value = pbs.read();
+                    if (value == -1) {
+                        throw new IOException("Unexpected End while searching for first Mime Boundary");
+                    }
+                    boundaryIndex++;
+                }
+                if (boundaryIndex == bp.length) {
+                    // boundary found
+                    pbs.read();
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    private Attachment readMimePart() throws MessagingException, IOException {
+
+        int v = stream.read();
+        if (v == -1) {
+            return null;
+        }
+        stream.unread(v);
+        InternetHeaders headers;
+        headers = new InternetHeaders(stream);
+        MimeBodyPartInputStream partStream = new MimeBodyPartInputStream(stream, boundary.getBytes());
+        final CachedOutputStream cos = new CachedOutputStream();
+        cos.setThreshold(THRESHHOLD);
+        AbstractCachedOutputStream.copyStream(partStream, cos, THRESHHOLD);
+        cos.close();
+        final String ct = headers.getHeader("Content-Type", null);
+        cache.add(cos);
+        DataSource source = new AttachmentDataSource(ct, cos);
+        DataHandler dh = new DataHandler(source);
+        String id = headers.getHeader("Content-ID", null);
+        if (id != null && id.startsWith("<")) {
+            id = id.substring(1, id.length() - 1);
+        }
+        AttachmentImpl att = new AttachmentImpl(id, dh);
+        for (Enumeration<?> e = headers.getAllHeaders(); e.hasMoreElements();) {
+            Header header = (Header) e.nextElement();
+            if (header.getName().equalsIgnoreCase("Content-Transfer-Encoding")
+                            && header.getValue().equalsIgnoreCase("binary")) {
+                att.setXOP(true);
+            }
+            att.setHeader(header.getName(), header.getValue());
+        }
+        return att;
+    }
+
+    private class MimeBodyPartInputStream extends InputStream {
+
+        PushbackInputStream inStream;
+
+        boolean boundaryFound;
+
+        byte[] boundary;
+
+        public MimeBodyPartInputStream(PushbackInputStream inStreamParam, byte[] boundaryParam) {
+            super();
+            this.inStream = inStreamParam;
+            this.boundary = boundaryParam;
+        }
+
+        public int read() throws IOException {
+            boolean needUnread0d0a = false;
+            if (boundaryFound) {
+                return -1;
+            }
+
+            // read the next value from stream
+            int value = inStream.read();
+            // A problem occured because all the mime parts tends to have a /r/n
+            // at the end. Making it hard to transform them to correct
+            // DataSources.
+            // This logic introduced to handle it
+            if (value == 13) {
+                value = inStream.read();
+                if (value != 10) {
+                    inStream.unread(value);
+                    return 13;
+                } else {
+                    value = inStream.read();
+                    if ((byte) value != boundary[0]) {
+                        inStream.unread(value);
+                        inStream.unread(10);
+                        return 13;
+                    } else {
+                        needUnread0d0a = true;
+                    }
+                }
+            } else if ((byte) value != boundary[0]) {
+                return value;
+            }
+            // read value is the first byte of the boundary. Start matching the
+            // next characters to find a boundary
+            int boundaryIndex = 0;
+            while ((boundaryIndex < boundary.length) && ((byte) value == boundary[boundaryIndex])) {
+                value = inStream.read();
+                boundaryIndex++;
+            }
+            if (boundaryIndex == boundary.length) {
+                // boundary found
+                boundaryFound = true;
+                // read the end of line character
+                if (inStream.read() == 45 && value == 45) {
+                    // Last mime boundary should have a succeeding "--"
+                    // as we are on it, read the terminating CRLF
+                    inStream.read();
+                    inStream.read();
+                }
+                return -1;
+            }
+            // Boundary not found. Restoring bytes skipped.
+            // write first skipped byte, push back the rest
+            if (value != -1) {
+                // Stream might have ended
+                inStream.unread(value);
+            }
+            if (needUnread0d0a) {
+                inStream.unread(boundary, 0, boundaryIndex);
+                inStream.unread(10);
+                value = 13;
+            } else {
+                inStream.unread(boundary, 1, boundaryIndex - 1);                
+                value = boundary[0];
+            }
+            return value;
+        }
+    }
+
+    protected static void printStream(InputStream in) throws IOException {
+        for (int i = in.read(); i != -1; i = in.read()) {
+            System.out.write(i);
+        }
+        System.out.println("print stream");
+    }
+}

Propchange: incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentDeserializer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentDeserializer.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentSerializer.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentSerializer.java?view=auto&rev=467624
==============================================================================
--- incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentSerializer.java (added)
+++ incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentSerializer.java Wed Oct 25 05:36:43 2006
@@ -0,0 +1,124 @@
+/**
+ * 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.cxf.binding.attachment;
+
+import java.io.InputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cxf.helpers.CastUtils;
+import org.apache.cxf.interceptor.Fault;
+import org.apache.cxf.io.AbstractCachedOutputStream;
+import org.apache.cxf.message.Attachment;
+import org.apache.cxf.message.Message;
+
+public class AttachmentSerializer {
+
+    private static final String LINE_SEP = System.getProperty("line.separator");
+    
+    private Message message;
+
+    private InputStream in;
+
+    private OutputStream out;
+
+    public AttachmentSerializer(Message messageParam, InputStream inParam, OutputStream outParam) {
+        message = messageParam;
+        in = inParam;
+        out = outParam;
+    }
+
+    /**
+     * Using result in soapMessage & attachment to write to output stream
+     * 
+     * @param soapMessage
+     * @param in
+     *            input stream contain the attachment
+     * @param out
+     * @throws CxfRioException
+     */
+
+    public String serializeMultipartMessage() {
+
+        String soapPartId;
+        String boundary = AttachmentUtil.getUniqueBoundaryValue(0);
+        try {
+            soapPartId = AttachmentUtil.createContentID(null);
+        } catch (UnsupportedEncodingException e) {
+            throw new Fault(e);
+        }
+        try {
+            Map<String, List<String>> headers = CastUtils.cast((Map<?, ?>) message
+                            .get(Message.PROTOCOL_HEADERS));
+            if (headers == null) {
+                // this is the case of server out (response)
+                headers = new HashMap<String, List<String>>();
+                message.put(Message.PROTOCOL_HEADERS, headers);
+            }
+            AttachmentUtil.setMimeRequestHeader(headers, message, soapPartId,
+                            "soap message with attachments", boundary);
+            //finish prepare header, call flush to flush headers and resetOut to get wire stream
+            out.flush();
+            
+            String soapHeader = AttachmentUtil.getSoapPartHeader(message, soapPartId, "");
+            out.write(("--" + boundary + LINE_SEP).getBytes());
+            out.write(soapHeader.getBytes());            
+            out.write(LINE_SEP.getBytes());            
+            AbstractCachedOutputStream.copyStream(in, out, 64 * 1024);
+            if (!System.getProperty("file.separator").equals("/")) {
+                out.write(LINE_SEP.getBytes());
+            }            
+            for (Attachment att : message.getAttachments()) {
+                soapHeader = AttachmentUtil.getAttchmentPartHeader(att);
+                out.write(("--" + boundary + LINE_SEP).getBytes());
+                out.write(soapHeader.getBytes());                
+                out.write(LINE_SEP.getBytes());                
+                Object content = att.getDataHandler().getContent();
+                if (content instanceof InputStream) {
+                    InputStream insAtt = (InputStream) content;
+                    AbstractCachedOutputStream.copyStream(insAtt, out, 64 * 1024);
+                } else {                    
+                    ObjectOutputStream oos = new ObjectOutputStream(out);
+                    oos.writeObject(content);
+                }
+                if (!System.getProperty("file.separator").equals("/")) {
+                    out.write(LINE_SEP.getBytes());
+                }
+            }
+            out.write(("--" + boundary + "--").getBytes());
+            out.write(LINE_SEP.getBytes());
+            out.flush();            
+            // build contentType string for return
+            List<String> contentType = (List<String>) headers.get("Content-Type");
+            StringBuffer sb = new StringBuffer(120);
+            for (String s : contentType) {
+                sb.append(s);
+            }
+            return sb.toString();
+        } catch (Exception e) {
+            throw new Fault(e);
+        }
+
+    }
+}

Propchange: incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentSerializer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/binding/attachment/AttachmentSerializer.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/interceptor/AbstractFaultChainIntiatorObserver.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/interceptor/AbstractFaultChainIntiatorObserver.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/interceptor/AbstractFaultChainIntiatorObserver.java (original)
+++ incubator/cxf/trunk/rt/core/src/main/java/org/apache/cxf/interceptor/AbstractFaultChainIntiatorObserver.java Wed Oct 25 05:36:43 2006
@@ -20,9 +20,12 @@
 package org.apache.cxf.interceptor;
 
 import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.cxf.Bus;
 import org.apache.cxf.binding.Binding;
+import org.apache.cxf.common.logging.LogUtils;
 import org.apache.cxf.message.Exchange;
 import org.apache.cxf.message.Message;
 import org.apache.cxf.message.MessageImpl;
@@ -31,6 +34,9 @@
 import org.apache.cxf.transport.MessageObserver;
 
 public abstract class AbstractFaultChainIntiatorObserver implements MessageObserver {
+    
+    private static final Logger LOG = Logger.getLogger(AbstractFaultChainIntiatorObserver.class.getName());
+    
     private Bus bus;
 
     public AbstractFaultChainIntiatorObserver(Bus bus) {
@@ -62,7 +68,11 @@
         initializeInterceptors(faultMessage.getExchange(), chain);
         
         faultMessage.setInterceptorChain(chain);
-        chain.doIntercept(faultMessage);
+        try {
+            chain.doIntercept(faultMessage);
+        } catch (Exception ex) {
+            LogUtils.log(LOG, Level.INFO, "Error occured during error handling, give up!", ex);
+        }
     }
 
     protected abstract List<Phase> getPhases();

Modified: incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/attachment/JAXBAttachmentUnmarshaller.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/attachment/JAXBAttachmentUnmarshaller.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/attachment/JAXBAttachmentUnmarshaller.java (original)
+++ incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/attachment/JAXBAttachmentUnmarshaller.java Wed Oct 25 05:36:43 2006
@@ -29,6 +29,7 @@
 import javax.mail.MessagingException;
 import javax.xml.bind.attachment.AttachmentUnmarshaller;
 
+import org.apache.cxf.binding.attachment.AttachmentDeserializer;
 import org.apache.cxf.common.logging.LogUtils;
 import org.apache.cxf.interceptor.Fault;
 import org.apache.cxf.message.Attachment;

Modified: incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/io/MessageDataReader.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/io/MessageDataReader.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/io/MessageDataReader.java (original)
+++ incubator/cxf/trunk/rt/databinding/jaxb/src/main/java/org/apache/cxf/jaxb/io/MessageDataReader.java Wed Oct 25 05:36:43 2006
@@ -26,10 +26,10 @@
 
 import org.w3c.dom.Node;
 
+import org.apache.cxf.binding.attachment.AttachmentDeserializer;
 import org.apache.cxf.databinding.DataReader;
 import org.apache.cxf.jaxb.JAXBDataReaderFactory;
 import org.apache.cxf.jaxb.JAXBEncoderDecoder;
-import org.apache.cxf.jaxb.attachment.AttachmentDeserializer;
 import org.apache.cxf.jaxb.attachment.JAXBAttachmentUnmarshaller;
 import org.apache.cxf.message.Message;
 

Modified: incubator/cxf/trunk/rt/frontend/jaxws/src/test/java/org/apache/cxf/jaxws/JaxWsClientTest.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/rt/frontend/jaxws/src/test/java/org/apache/cxf/jaxws/JaxWsClientTest.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/rt/frontend/jaxws/src/test/java/org/apache/cxf/jaxws/JaxWsClientTest.java (original)
+++ incubator/cxf/trunk/rt/frontend/jaxws/src/test/java/org/apache/cxf/jaxws/JaxWsClientTest.java Wed Oct 25 05:36:43 2006
@@ -117,9 +117,8 @@
             client.getEndpoint().getOutInterceptors().add(new FaultThrower());
             client.invoke(bop, new Object[] {"BadRecordLitFault"}, null);
             fail("Should have returned a fault!");
-        } catch (Fault fault) {
-            assertEquals("Foo", fault.getCode());
-            assertEquals("Foo", fault.getMessage());
+        } catch (RuntimeException ex) {
+            assertEquals(true, ex.getMessage().indexOf("Foo") > 0);
         } 
     }
 

Modified: incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/mtom/TestMultipartMessageInterceptor.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/mtom/TestMultipartMessageInterceptor.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/mtom/TestMultipartMessageInterceptor.java (original)
+++ incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/mtom/TestMultipartMessageInterceptor.java Wed Oct 25 05:36:43 2006
@@ -21,10 +21,10 @@
 
 import junit.framework.TestCase;
 
+import org.apache.cxf.binding.attachment.AttachmentDeserializer;
 import org.apache.cxf.binding.soap.SoapMessage;
 import org.apache.cxf.binding.soap.interceptor.MultipartMessageInterceptor;
 import org.apache.cxf.interceptor.Fault;
-import org.apache.cxf.jaxb.attachment.AttachmentDeserializer;
 
 public class TestMultipartMessageInterceptor extends MultipartMessageInterceptor {
 

Modified: incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/type_test/AbstractTypeTestClient.java
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/type_test/AbstractTypeTestClient.java?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/type_test/AbstractTypeTestClient.java (original)
+++ incubator/cxf/trunk/systests/src/test/java/org/apache/cxf/systest/type_test/AbstractTypeTestClient.java Wed Oct 25 05:36:43 2006
@@ -30,7 +30,6 @@
 import javax.xml.datatype.XMLGregorianCalendar;
 import javax.xml.namespace.QName;
 import javax.xml.ws.Holder;
-//import javax.xml.ws.WebServiceException;
 
 import org.apache.cxf.systest.common.ClientServerTestBase;
 import org.apache.type_test.doc.TypeTestPortType;
@@ -596,10 +595,9 @@
             }
         }
     }
-    public void testDate() throws Exception {
-        assertEquals(true, true);
-    }
-    /* Revisit When client Fault is ready. Comment should be removed
+
+    // Revisit When client Fault is ready. Comment should be removed
+    
     public void testDate() throws Exception {
         javax.xml.datatype.DatatypeFactory datatypeFactory = javax.xml.datatype.DatatypeFactory.newInstance();
 
@@ -645,10 +643,11 @@
             }
             fail("Expected to catch WebServiceException when calling"
                  + " testDate() with uninitialized parameters.");
-        } catch (WebServiceException e) {
-            // Ignore expected //failure.
+        } catch (RuntimeException re) {
+            assertNotNull(re);
         }
-    }*/
+    }
+    
 
     public void testDateTime() throws Exception {
         javax.xml.datatype.DatatypeFactory datatypeFactory = javax.xml.datatype.DatatypeFactory.newInstance();

Modified: incubator/cxf/trunk/testutils/pom.xml
URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/testutils/pom.xml?view=diff&rev=467624&r1=467623&r2=467624
==============================================================================
--- incubator/cxf/trunk/testutils/pom.xml (original)
+++ incubator/cxf/trunk/testutils/pom.xml Wed Oct 25 05:36:43 2006
@@ -303,6 +303,11 @@
                                 <wsdlOption>
                                     <wsdl>${basedir}/src/main/resources/wsdl/addNumbers.wsdl</wsdl>
                                 </wsdlOption>
+
+                                <wsdlOption>
+                                    <wsdl>${basedir}/src/main/resources/wsdl/addNumbers_2int.wsdl</wsdl>
+                                </wsdlOption>
+
                             </wsdlOptions>
                         </configuration>
                         <goals>



Re: svn commit: r467624 - in /incubator/cxf/trunk: api/src/main/java/org/apache/cxf/phase/ rt/bindings/soap/src/main/java/org/apache/cxf/binding/soap/interceptor/ rt/bindings/soap/src/test/java/org/apache/cxf/binding/soap/attachments/ rt/core/ rt/core/src/...

Posted by Dan Diephouse <da...@envoisolutions.com>.
Hi Tom,

Can you explain why we're doing this here? I'm not sure that the chain 
should depend on knowledge about whether a client is running it or not. 
Also, if an Exception happens I think it is the client's responsibility 
to check for a message.getContent(Exception.class) - I thought we were 
already doing so even! Can you please look at changing this back so we 
aren't rethrowing anything?

Thanks,
- Dan

tli@apache.org wrote:

>Modified: incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java
>URL: http://svn.apache.org/viewvc/incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java?view=diff&rev=467624&r1=467623&r2=467624
>==============================================================================
>--- incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java (original)
>+++ incubator/cxf/trunk/api/src/main/java/org/apache/cxf/phase/PhaseInterceptorChain.java Wed Oct 25 05:36:43 2006
>@@ -154,6 +154,13 @@
>                 
>                 if (faultObserver != null) {
>                     faultObserver.onMessage(message);
>+                } else {
>+                    // Client out-bound message, directly throw exception back to client
>+                    if (message.getExchange() != null 
>+                            && message == message.getExchange().getOutMessage() 
>+                            && message.containsKey(Message.REQUESTOR_ROLE)) {
>+                        throw new RuntimeException(ex);
>+                    }
>                 }
>                 state = State.ABORTED;
>             } 
>
>
>  
>


-- 
Dan Diephouse
(616) 971-2053
Envoi Solutions LLC
http://netzooid.com