You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by js...@apache.org on 2007/10/10 11:17:55 UTC

svn commit: r583400 - in /activemq/camel/trunk: camel-core/src/main/java/org/apache/camel/builder/ camel-core/src/main/java/org/apache/camel/model/ camel-core/src/main/java/org/apache/camel/model/dataformat/ camel-core/src/main/java/org/apache/camel/pr...

Author: jstrachan
Date: Wed Oct 10 02:17:53 2007
New Revision: 583400

URL: http://svn.apache.org/viewvc?rev=583400&view=rev
Log:
working dataformat marshalling and unmarshalling for Serialization and JAXB2 for CAMEL-165

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DataTypeExpression.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MarshalProcessor.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/UnmarshalProcessor.java   (with props)
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DataFormatTest.java   (with props)
    activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/DataFormatTest.java   (with props)
    activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/UnmarshalTest.java   (with props)
    activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/
    activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/
    activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/camel/
    activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/camel/example/
    activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/camel/example/jaxb.index
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/MarshalType.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/UnmarshalType.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/ArtixDSDataFormat.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/DataFormatType.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/JaxbDataFormat.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/SerializationDataFormat.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/XMLBeansDataFormat.java
    activemq/camel/trunk/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
    activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/PurchaseOrder.java

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DataTypeExpression.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DataTypeExpression.java?rev=583400&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DataTypeExpression.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DataTypeExpression.java Wed Oct 10 02:17:53 2007
@@ -0,0 +1,95 @@
+/**
+ *
+ * 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.builder;
+
+import org.apache.camel.model.ProcessorType;
+import org.apache.camel.model.dataformat.JaxbDataFormat;
+import org.apache.camel.model.dataformat.DataFormatType;
+import org.apache.camel.model.dataformat.SerializationDataFormat;
+import org.apache.camel.model.dataformat.XMLBeansDataFormat;
+import org.apache.camel.model.dataformat.ArtixDSDataFormat;
+import org.apache.camel.spi.DataFormat;
+
+/**
+ * An expression for constructing the different possible {@link DataFormat}
+ * options.
+ *
+ * @version $Revision: 1.1 $
+ */
+public class DataTypeExpression<T extends ProcessorType> {
+    private final ProcessorType<T> processorType;
+    private final Operation operation;
+
+    public enum Operation {
+        Marshal, Unmarshal
+    };
+
+    public DataTypeExpression(ProcessorType<T> processorType, Operation operation) {
+        this.processorType = processorType;
+        this.operation = operation;
+    }
+
+    /**
+     * Uses the Java Serialization data format
+     */
+    public T serialization() {
+        return dataFormat(new SerializationDataFormat());
+    }
+
+    /**
+     * Uses the JAXB data format
+     */
+    public T jaxb() {
+        return dataFormat(new JaxbDataFormat());
+    }
+
+    /**
+     * Uses the JAXB data format turning pretty printing on or off
+     */
+    public T jaxb(boolean prettyPrint) {
+        return dataFormat(new JaxbDataFormat(prettyPrint));
+    }
+
+    /**
+     * Uses the JAXB data format
+     */
+    public T xmlBeans() {
+        return dataFormat(new XMLBeansDataFormat());
+    }
+
+    /**
+     * Uses the
+     * <a href="http://www.iona.com/artix/ds">Artix Data Services</a>
+     * data format for dealing with lots of different message formats such as SWIFT etc.
+     */
+    public T artixDS() {
+        return dataFormat(new ArtixDSDataFormat());
+    }
+
+    private T dataFormat(DataFormatType dataFormatType) {
+        switch (operation) {
+            case Unmarshal:
+                return processorType.unmarshal(dataFormatType);
+            case Marshal:
+                return processorType.marshal(dataFormatType);
+            default:
+                throw new IllegalArgumentException("Unknown value: " + operation);
+        }
+    }
+
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/builder/DataTypeExpression.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/MarshalType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/MarshalType.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/MarshalType.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/MarshalType.java Wed Oct 10 02:17:53 2007
@@ -23,6 +23,8 @@
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Processor;
+import org.apache.camel.processor.MarshalProcessor;
+import org.apache.camel.spi.DataFormat;
 import org.apache.camel.impl.RouteContext;
 import org.apache.camel.model.dataformat.DataFormatType;
 import static org.apache.camel.util.ObjectHelper.notNull;
@@ -43,6 +45,10 @@
     public MarshalType() {
     }
 
+    public MarshalType(DataFormatType dataFormatType) {
+        this.dataFormatType = dataFormatType;
+    }
+
     @Override
     public String toString() {
         if (dataFormatType != null) {
@@ -76,6 +82,7 @@
             notNull(ref, "ref or dataFormatType");
             type = routeContext.lookup(ref, DataFormatType.class);
         }
-        throw new UnsupportedOperationException("Not implemented yet!");
+        DataFormat dataFormat = type.getDataFormat(routeContext);
+        return new MarshalProcessor(dataFormat);
     }
 }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/ProcessorType.java Wed Oct 10 02:17:53 2007
@@ -37,8 +37,10 @@
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.builder.NoErrorHandlerBuilder;
 import org.apache.camel.builder.ProcessorBuilder;
+import org.apache.camel.builder.DataTypeExpression;
 import org.apache.camel.converter.ObjectConverter;
 import org.apache.camel.impl.RouteContext;
+import org.apache.camel.model.dataformat.DataFormatType;
 import org.apache.camel.model.language.ExpressionType;
 import org.apache.camel.model.language.LanguageExpression;
 import org.apache.camel.processor.DelegateProcessor;
@@ -48,6 +50,7 @@
 import org.apache.camel.processor.aggregate.AggregationStrategy;
 import org.apache.camel.processor.idempotent.IdempotentConsumer;
 import org.apache.camel.processor.idempotent.MessageIdRepository;
+import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.Policy;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -724,6 +727,35 @@
     public Type convertFaultBodyTo(Class type) {
         return process(ProcessorBuilder.setFaultBody(Builder.faultBody().convertTo(type)));
     }
+
+    // DataFormat support
+    // -------------------------------------------------------------------------
+    public DataTypeExpression<Type> unmarshal() {
+        return new DataTypeExpression<Type>(this, DataTypeExpression.Operation.Unmarshal);
+    }
+
+    public Type unmarshal(DataFormatType dataFormatType) {
+        addOutput(new UnmarshalType(dataFormatType));
+        return (Type) this;
+    }
+
+    public Type unmarshal(DataFormat dataFormat) {
+        return unmarshal(new DataFormatType(dataFormat));
+    }
+
+    public DataTypeExpression<Type> marshal() {
+        return new DataTypeExpression<Type>(this, DataTypeExpression.Operation.Marshal);
+    }
+
+    public Type marshal(DataFormatType dataFormatType) {
+        addOutput(new MarshalType(dataFormatType));
+        return (Type) this;
+    }
+
+    public Type marshal(DataFormat dataFormat) {
+        return marshal(new DataFormatType(dataFormat));
+    }
+
 
     // Properties
     // -------------------------------------------------------------------------

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/UnmarshalType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/UnmarshalType.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/UnmarshalType.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/UnmarshalType.java Wed Oct 10 02:17:53 2007
@@ -25,6 +25,8 @@
 import org.apache.camel.Processor;
 import org.apache.camel.impl.RouteContext;
 import org.apache.camel.model.dataformat.DataFormatType;
+import org.apache.camel.processor.UnmarshalProcessor;
+import org.apache.camel.spi.DataFormat;
 import static org.apache.camel.util.ObjectHelper.notNull;
 
 /**
@@ -43,6 +45,10 @@
     public UnmarshalType() {
     }
 
+    public UnmarshalType(DataFormatType dataFormatType) {
+        this.dataFormatType = dataFormatType;
+    }
+
     @Override
     public String toString() {
         if (dataFormatType != null) {
@@ -76,6 +82,7 @@
             notNull(ref, "ref or dataFormatType");
             type = routeContext.lookup(ref, DataFormatType.class);
         }
-        throw new UnsupportedOperationException("Not implemented yet!");
+        DataFormat dataFormat = type.getDataFormat(routeContext);
+        return new UnmarshalProcessor(dataFormat);
     }
 }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/ArtixDSDataFormat.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/ArtixDSDataFormat.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/ArtixDSDataFormat.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/ArtixDSDataFormat.java Wed Oct 10 02:17:53 2007
@@ -25,7 +25,8 @@
 import org.apache.camel.spi.DataFormat;
 
 /**
- * Represents the Artix Data Services {@link DataFormat}
+ * Represents the <a href="http://www.iona.com/artix/ds">Artix Data Services</a>
+ * {@link DataFormat}
  *
  * @version $Revision: 1.1 $
  */
@@ -36,6 +37,10 @@
     private String element;
     @XmlAttribute
     private String format;
+
+    public ArtixDSDataFormat() {
+        super("org.apache.camel.artix.ds.ArtixDSDataFormat");
+    }
 
     public String getElement() {
         return element;

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/DataFormatType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/DataFormatType.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/DataFormatType.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/DataFormatType.java Wed Oct 10 02:17:53 2007
@@ -19,12 +19,52 @@
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
+import org.apache.camel.impl.RouteContext;
+import org.apache.camel.spi.DataFormat;
+import org.apache.camel.util.ObjectHelper;
+
 /**
  * @version $Revision: 1.1 $
  */
 @XmlType(name = "dataFormatType")
 @XmlAccessorType(XmlAccessType.FIELD)
-public abstract class DataFormatType {
+public class DataFormatType {
+    @XmlTransient
+    private DataFormat dataFormat;
+    @XmlTransient
+    private String dataFormatTypeName;
+
+    public DataFormatType() {
+    }
+
+    public DataFormatType(DataFormat dataFormat) {
+        this.dataFormat = dataFormat;
+    }
+
+    protected DataFormatType(String dataFormatTypeName) {
+        this.dataFormatTypeName = dataFormatTypeName;
+    }
+
+    public DataFormat getDataFormat(RouteContext routeContext) {
+        if (dataFormat == null) {
+            dataFormat = createDataFormat(routeContext);
+            ObjectHelper.notNull(dataFormat, "dataFormat");
+        }
+        return dataFormat;
+    }
+
+    protected DataFormat createDataFormat(RouteContext routeContext) {
+        if (dataFormatTypeName != null) {
+            Class type = ObjectHelper.loadClass(dataFormatTypeName, getClass().getClassLoader());
+            if (type == null) {
+                throw new IllegalArgumentException("The class " + dataFormatTypeName + " is not on the classpath! Cannot use the dataFormat " + this);
+            }
+            return (DataFormat) ObjectHelper.newInstance(type);
+        }
+        return null;
+    }
+
 }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/JaxbDataFormat.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/JaxbDataFormat.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/JaxbDataFormat.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/JaxbDataFormat.java Wed Oct 10 02:17:53 2007
@@ -35,6 +35,15 @@
     @XmlAttribute(required = false)
     private Boolean prettyPrint;
 
+    public JaxbDataFormat() {
+        super("org.apache.camel.converter.jaxb.JaxbDataFormat");
+    }
+
+    public JaxbDataFormat(boolean prettyPrint) {
+        this();
+        setPrettyPrint(prettyPrint);
+    }
+
     public Boolean getPrettyPrint() {
         return prettyPrint;
     }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/SerializationDataFormat.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/SerializationDataFormat.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/SerializationDataFormat.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/SerializationDataFormat.java Wed Oct 10 02:17:53 2007
@@ -20,6 +20,7 @@
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.spi.DataFormat;
+import org.apache.camel.impl.RouteContext;
 
 /**
  * Represents the Java Serialization {@link DataFormat}
@@ -28,4 +29,8 @@
  */
 @XmlRootElement(name = "serialization")
 public class SerializationDataFormat extends DataFormatType {
+    @Override
+    protected DataFormat createDataFormat(RouteContext routeContext) {
+        return new org.apache.camel.impl.SerializationDataFormat();
+    }
 }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/XMLBeansDataFormat.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/XMLBeansDataFormat.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/XMLBeansDataFormat.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/dataformat/XMLBeansDataFormat.java Wed Oct 10 02:17:53 2007
@@ -35,6 +35,10 @@
     @XmlAttribute(required = false)
     private Boolean prettyPrint;
 
+    public XMLBeansDataFormat() {
+        super("org.apache.camel.dataformat.converter.XmlBeansDataType");
+    }
+
     public Boolean getPrettyPrint() {
         return prettyPrint;
     }

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MarshalProcessor.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MarshalProcessor.java?rev=583400&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MarshalProcessor.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MarshalProcessor.java Wed Oct 10 02:17:53 2007
@@ -0,0 +1,49 @@
+/**
+ *
+ * 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.processor;
+
+import java.io.ByteArrayOutputStream;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.Message;
+import org.apache.camel.spi.DataFormat;
+
+/**
+ * Marshals the body of the incoming message using the given data format
+ *
+ * @version $Revision: 1.1 $
+ */
+public class MarshalProcessor implements Processor {
+    private final DataFormat dataFormat;
+
+    public MarshalProcessor(DataFormat dataFormat) {
+        this.dataFormat = dataFormat;
+    }
+
+    public void process(Exchange exchange) throws Exception {
+        ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+        Message in = exchange.getIn();
+        Object body = in.getBody();
+        dataFormat.marshal(exchange, body, buffer);
+        byte[] data = buffer.toByteArray();
+        Message out = exchange.getOut(true);
+        out.copyFrom(in);
+        out.setBody(data);
+    }
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/MarshalProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/UnmarshalProcessor.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/UnmarshalProcessor.java?rev=583400&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/UnmarshalProcessor.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/UnmarshalProcessor.java Wed Oct 10 02:17:53 2007
@@ -0,0 +1,47 @@
+/**
+ *
+ * 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.processor;
+
+import java.io.InputStream;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.Message;
+import org.apache.camel.spi.DataFormat;
+import org.apache.camel.util.ExchangeHelper;
+
+/**
+ * Unmarshals the body of the incoming message using the given data format
+ *
+ * @version $Revision: 1.1 $
+ */
+public class UnmarshalProcessor implements Processor {
+    private final DataFormat dataFormat;
+
+    public UnmarshalProcessor(DataFormat dataFormat) {
+        this.dataFormat = dataFormat;
+    }
+
+    public void process(Exchange exchange) throws Exception {
+        InputStream stream = ExchangeHelper.getMandatoryInBody(exchange, InputStream.class);
+        Object result = dataFormat.unmarshal(exchange, stream);
+        Message out = exchange.getOut(true);
+        out.copyFrom(exchange.getIn());
+        out.setBody(result);
+    }
+}
\ No newline at end of file

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/UnmarshalProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DataFormatTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DataFormatTest.java?rev=583400&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DataFormatTest.java (added)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DataFormatTest.java Wed Oct 10 02:17:53 2007
@@ -0,0 +1,68 @@
+/**
+ * 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.processor;
+
+import java.io.Serializable;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.util.ObjectHelper;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class DataFormatTest extends ContextTestSupport {
+
+    public void testMarshalThenUnmarshalBean() throws Exception {
+        MyBean bean = new MyBean();
+        bean.name = "James";
+        bean.counter = 5;
+
+        MockEndpoint resultEndpoint = resolveMandatoryEndpoint("mock:result", MockEndpoint.class);
+        resultEndpoint.expectedBodiesReceived(bean);
+
+        template.sendBody("direct:start", bean);
+
+        resultEndpoint.assertIsSatisfied();
+    }
+
+
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                from("direct:start").marshal().serialization().to("direct:marshalled");
+                from("direct:marshalled").unmarshal().serialization().to("mock:result");
+            }
+        };
+    }
+
+    protected static class MyBean implements Serializable {
+        public String name;
+        public int counter;
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof MyBean) {
+                MyBean that = (MyBean) o;
+                return ObjectHelper.equals(this.name, that.name) && ObjectHelper.equals(this.counter,  that.counter);                
+            }
+            return false;
+        }
+    }
+
+}
\ No newline at end of file

Propchange: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/DataFormatTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java (original)
+++ activemq/camel/trunk/components/camel-jaxb/src/main/java/org/apache/camel/converter/jaxb/JaxbDataFormat.java Wed Oct 10 02:17:53 2007
@@ -32,11 +32,12 @@
 
 /**
  * A {@link DataFormat} using JAXB2 to marshal to and from XML
- * 
+ *
  * @version $Revision: 1.1 $
  */
 public class JaxbDataFormat implements DataFormat {
     private JAXBContext context;
+    private String contextPath;
     private boolean prettyPrint = true;
     private Marshaller marshaller;
     private Unmarshaller unmarshaller;
@@ -48,9 +49,13 @@
         this.context = context;
     }
 
+    public JaxbDataFormat(String contextPath) {
+        this.contextPath = contextPath;
+    }
+
     public void marshal(Exchange exchange, Object graph, OutputStream stream) throws IOException {
         try {
-            marshaller.marshal(graph, stream);
+            getMarshaller().marshal(graph, stream);
         }
         catch (JAXBException e) {
             throw IOHelper.createIOException(e);
@@ -59,7 +64,7 @@
 
     public Object unmarshal(Exchange exchange, InputStream stream) throws IOException, ClassNotFoundException {
         try {
-            return unmarshaller.unmarshal(stream);
+            return getUnmarshaller().unmarshal(stream);
         }
         catch (JAXBException e) {
             throw IOHelper.createIOException(e);
@@ -110,6 +115,11 @@
     }
 
     protected JAXBContext createContext() throws JAXBException {
-        return JAXBContext.newInstance();
+        if (contextPath != null) {
+            return JAXBContext.newInstance(contextPath);
+        }
+        else {
+            return JAXBContext.newInstance();
+        }
     }
 }

Added: activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/DataFormatTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/DataFormatTest.java?rev=583400&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/DataFormatTest.java (added)
+++ activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/DataFormatTest.java Wed Oct 10 02:17:53 2007
@@ -0,0 +1,62 @@
+/**
+ * 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.example;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.converter.jaxb.JaxbDataFormat;
+import org.apache.camel.spi.DataFormat;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class DataFormatTest extends ContextTestSupport {
+
+    public void testMarshalThenUnmarshalBean() throws Exception {
+        PurchaseOrder bean = new PurchaseOrder();
+        bean.setName("Beer");
+        bean.setAmount(23);
+        bean.setPrice(2.5);
+
+        MockEndpoint resultEndpoint = resolveMandatoryEndpoint("mock:result", MockEndpoint.class);
+        resultEndpoint.expectedBodiesReceived(bean);
+
+        template.sendBody("direct:start", bean);
+
+        resultEndpoint.assertIsSatisfied();
+    }
+
+
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+
+                DataFormat jaxb = new JaxbDataFormat("org.apache.camel.example");
+
+                from("direct:start").
+                        marshal(jaxb).
+                        to("direct:marshalled");
+
+                from("direct:marshalled").
+                        unmarshal(jaxb).
+                        to("mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

Propchange: activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/DataFormatTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/PurchaseOrder.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/PurchaseOrder.java?rev=583400&r1=583399&r2=583400&view=diff
==============================================================================
--- activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/PurchaseOrder.java (original)
+++ activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/PurchaseOrder.java Wed Oct 10 02:17:53 2007
@@ -21,6 +21,8 @@
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 
+import org.apache.camel.util.ObjectHelper;
+
 /**
  * @version $Revision$
  */
@@ -39,6 +41,17 @@
         return "PurchaseOrder[name: " + name + " amount: " + amount + " price: " + price + "]";
     }
 
+    @Override
+    public boolean equals(Object o) {
+        if (o instanceof PurchaseOrder) {
+            PurchaseOrder that = (PurchaseOrder) o;
+            return ObjectHelper.equals(this.name, that.name) &&
+                    ObjectHelper.equals(this.amount, that.amount) &&
+                    ObjectHelper.equals(this.price, that.price);
+        }
+        return false;
+    }
+
     public double getAmount() {
         return amount;
     }
@@ -62,5 +75,4 @@
     public void setPrice(double price) {
         this.price = price;
     }
-
 }

Added: activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/UnmarshalTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/UnmarshalTest.java?rev=583400&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/UnmarshalTest.java (added)
+++ activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/UnmarshalTest.java Wed Oct 10 02:17:53 2007
@@ -0,0 +1,58 @@
+/**
+ * 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.example;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.converter.jaxb.JaxbDataFormat;
+import org.apache.camel.spi.DataFormat;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class UnmarshalTest extends ContextTestSupport {
+
+    public void testSendXmlAndUnmarshal() throws Exception {
+        PurchaseOrder expected = new PurchaseOrder();
+        expected.setName("Wine");
+        expected.setAmount(123.45);
+        expected.setPrice(2.22);
+
+        MockEndpoint resultEndpoint = resolveMandatoryEndpoint("mock:result", MockEndpoint.class);
+        resultEndpoint.expectedBodiesReceived(expected);
+
+        template.sendBody("direct:start", "<purchaseOrder name='Wine' amount='123.45' price='2.22'/>");
+
+        resultEndpoint.assertIsSatisfied();
+    }
+
+
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+
+                DataFormat jaxb = new JaxbDataFormat("org.apache.camel.example");
+
+                from("direct:start").
+                        unmarshal(jaxb).
+                        to("mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

Propchange: activemq/camel/trunk/components/camel-jaxb/src/test/java/org/apache/camel/example/UnmarshalTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/camel/example/jaxb.index
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/camel/example/jaxb.index?rev=583400&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/camel/example/jaxb.index (added)
+++ activemq/camel/trunk/components/camel-jaxb/src/test/resources/org/apache/camel/example/jaxb.index Wed Oct 10 02:17:53 2007
@@ -0,0 +1 @@
+PurchaseOrder
\ No newline at end of file