You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by zr...@apache.org on 2017/04/06 10:03:08 UTC

camel git commit: CAMEL-11115 Enhance binding support in RestComp...

Repository: camel
Updated Branches:
  refs/heads/master d1ee73ec8 -> 994bea0cf


CAMEL-11115 Enhance binding support in RestComp...

...onent

Added property `bindingMode` on `RestEndpoint` to specify binding mode
at endpoint level. Removed `type` property on
`RestProducerBindingProcessor` set from `inType`.


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

Branch: refs/heads/master
Commit: 994bea0cfe19a4099d7e5631e48b731045fb0839
Parents: d1ee73e
Author: Zoran Regvart <zr...@apache.org>
Authored: Thu Apr 6 12:00:15 2017 +0200
Committer: Zoran Regvart <zr...@apache.org>
Committed: Thu Apr 6 12:01:10 2017 +0200

----------------------------------------------------------------------
 camel-core/src/main/docs/rest-component.adoc    |   3 +-
 .../camel/component/rest/RestEndpoint.java      |  19 +++
 .../camel/component/rest/RestProducer.java      |  11 +-
 .../rest/RestProducerBindingProcessor.java      |   4 +-
 .../camel/component/rest/RestEndpointTest.java  |  67 ++++++++
 .../rest/RestProducerBindingProcessorTest.java  | 171 +++++++++++++++++++
 6 files changed, 266 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/994bea0c/camel-core/src/main/docs/rest-component.adoc
----------------------------------------------------------------------
diff --git a/camel-core/src/main/docs/rest-component.adoc b/camel-core/src/main/docs/rest-component.adoc
index 0bbce46..0e9516a 100644
--- a/camel-core/src/main/docs/rest-component.adoc
+++ b/camel-core/src/main/docs/rest-component.adoc
@@ -49,7 +49,7 @@ with the following path and query parameters:
 | **uriTemplate** | The uri template |  | String
 |=======================================================================
 
-#### Query Parameters (14 parameters):
+#### Query Parameters (15 parameters):
 
 [width="100%",cols="2,5,^1,2",options="header"]
 |=======================================================================
@@ -65,6 +65,7 @@ with the following path and query parameters:
 | **exceptionHandler** (consumer) | To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this options is not in use. By default the consumer will deal with exceptions that will be logged at WARN/ERROR level and ignored. |  | ExceptionHandler
 | **exchangePattern** (consumer) | Sets the default exchange pattern when creating an exchange. |  | ExchangePattern
 | **apiDoc** (producer) | The swagger api doc resource to use. The resource is loaded from classpath by default and must be in JSon format. |  | String
+| **bindingMode** (producer) | Configures the binding mode for the producer. If set to anything other than 'off' the producer will try to convert the body of the incoming message from inType to the json or xml and the response from json or xml to outType. |  | RestBindingMode
 | **host** (producer) | Host and port of HTTP service to use (override host in swagger schema) |  | String
 | **queryParameters** (producer) | Query parameters for the HTTP service to call |  | String
 | **synchronous** (advanced) | Sets whether synchronous processing should be strictly used or Camel is allowed to use asynchronous processing (if supported). | false | boolean

http://git-wip-us.apache.org/repos/asf/camel/blob/994bea0c/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
index 472bc0e..e4c95fa 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestEndpoint.java
@@ -27,6 +27,8 @@ import org.apache.camel.NoSuchBeanException;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
 import org.apache.camel.impl.DefaultEndpoint;
+import org.apache.camel.model.rest.RestBindingMode;
+import org.apache.camel.processor.UnmarshalProcessor;
 import org.apache.camel.spi.FactoryFinder;
 import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.RestConfiguration;
@@ -79,6 +81,8 @@ public class RestEndpoint extends DefaultEndpoint {
     private String host;
     @UriParam(label = "producer", multiValue = true)
     private String queryParameters;
+    @UriParam(label = "producer")
+    private RestBindingMode bindingMode;
 
     private Map<String, Object> parameters;
 
@@ -251,6 +255,20 @@ public class RestEndpoint extends DefaultEndpoint {
         this.queryParameters = queryParameters;
     }
 
+    public RestBindingMode getBindingMode() {
+        return bindingMode;
+    }
+
+    /**
+     * Configures the binding mode for the producer. If set to anything
+     * other than 'off' the producer will try to convert the body of
+     * the incoming message from inType to the json or xml, and the
+     * response from json or xml to outType.
+     */
+    public void setBindingMode(final RestBindingMode bindingMode) {
+        this.bindingMode = bindingMode;
+    }
+
     @Override
     public Producer createProducer() throws Exception {
         RestProducerFactory apiDocFactory = null;
@@ -353,6 +371,7 @@ public class RestEndpoint extends DefaultEndpoint {
             RestProducer answer = new RestProducer(this, producer, config);
             answer.setOutType(outType);
             answer.setType(inType);
+            answer.setBindingMode(bindingMode);
 
             return answer;
         } else {

http://git-wip-us.apache.org/repos/asf/camel/blob/994bea0c/camel-core/src/main/java/org/apache/camel/component/rest/RestProducer.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestProducer.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestProducer.java
index 644025f..860308c 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestProducer.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestProducer.java
@@ -33,6 +33,7 @@ import org.apache.camel.Exchange;
 import org.apache.camel.Message;
 import org.apache.camel.Producer;
 import org.apache.camel.impl.DefaultAsyncProducer;
+import org.apache.camel.model.rest.RestBindingMode;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.util.AsyncProcessorConverterHelper;
@@ -56,7 +57,7 @@ public class RestProducer extends DefaultAsyncProducer {
     private final CamelContext camelContext;
     private final RestConfiguration configuration;
     private boolean prepareUriTemplate = true;
-    private String bindingMode;
+    private RestBindingMode bindingMode;
     private Boolean skipBindingOnErrorCode;
     private String type;
     private String outType;
@@ -107,11 +108,11 @@ public class RestProducer extends DefaultAsyncProducer {
         this.prepareUriTemplate = prepareUriTemplate;
     }
 
-    public String getBindingMode() {
+    public RestBindingMode getBindingMode() {
         return bindingMode;
     }
 
-    public void setBindingMode(String bindingMode) {
+    public void setBindingMode(final RestBindingMode bindingMode) {
         this.bindingMode = bindingMode;
     }
 
@@ -236,7 +237,7 @@ public class RestProducer extends DefaultAsyncProducer {
         // these options can be overridden per endpoint
         String mode = configuration.getBindingMode().name();
         if (bindingMode != null) {
-            mode = bindingMode;
+            mode = bindingMode.name();
         }
         boolean skip = configuration.isSkipBindingOnErrorCode();
         if (skipBindingOnErrorCode != null) {
@@ -340,7 +341,7 @@ public class RestProducer extends DefaultAsyncProducer {
             setAdditionalConfiguration(configuration, camelContext, outJaxb, "xml.out.");
         }
 
-        return new RestProducerBindingProcessor(producer, camelContext, json, jaxb, outJson, outJaxb, mode, skip, type, outType);
+        return new RestProducerBindingProcessor(producer, camelContext, json, jaxb, outJson, outJaxb, mode, skip, outType);
     }
 
     private void setAdditionalConfiguration(RestConfiguration config, CamelContext context,

http://git-wip-us.apache.org/repos/asf/camel/blob/994bea0c/camel-core/src/main/java/org/apache/camel/component/rest/RestProducerBindingProcessor.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/rest/RestProducerBindingProcessor.java b/camel-core/src/main/java/org/apache/camel/component/rest/RestProducerBindingProcessor.java
index d6d0d14..eeb2e83 100644
--- a/camel-core/src/main/java/org/apache/camel/component/rest/RestProducerBindingProcessor.java
+++ b/camel-core/src/main/java/org/apache/camel/component/rest/RestProducerBindingProcessor.java
@@ -51,14 +51,13 @@ public class RestProducerBindingProcessor extends DelegateAsyncProcessor {
     private final AsyncProcessor xmlMarshal;
     private final String bindingMode;
     private final boolean skipBindingOnErrorCode;
-    private final String type;
     private final String outType;
 
     public RestProducerBindingProcessor(AsyncProcessor processor, CamelContext camelContext,
                                         DataFormat jsonDataFormat, DataFormat xmlDataFormat,
                                         DataFormat outJsonDataFormat, DataFormat outXmlDataFormat,
                                         String bindingMode, boolean skipBindingOnErrorCode,
-                                        String type, String outType) {
+                                        String outType) {
 
         super(processor);
 
@@ -88,7 +87,6 @@ public class RestProducerBindingProcessor extends DelegateAsyncProcessor {
 
         this.bindingMode = bindingMode;
         this.skipBindingOnErrorCode = skipBindingOnErrorCode;
-        this.type = type;
         this.outType = outType;
     }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/994bea0c/camel-core/src/test/java/org/apache/camel/component/rest/RestEndpointTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/RestEndpointTest.java b/camel-core/src/test/java/org/apache/camel/component/rest/RestEndpointTest.java
new file mode 100644
index 0000000..7cebed0
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/RestEndpointTest.java
@@ -0,0 +1,67 @@
+/**
+ * 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.component.rest;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Endpoint;
+import org.apache.camel.Producer;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.apache.camel.impl.DefaultComponent;
+import org.apache.camel.model.rest.RestBindingMode;
+import org.apache.camel.spi.RestProducerFactory;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RestEndpointTest {
+
+    public static class MockRest extends DefaultComponent implements RestProducerFactory {
+        @Override
+        public Producer createProducer(CamelContext camelContext, String host, String verb, String basePath,
+            String uriTemplate, String queryParameters, String consumes, String produces,
+            Map<String, Object> parameters) throws Exception {
+            return null;
+        }
+
+        @Override
+        protected Endpoint createEndpoint(String uri, String remaining, Map<String, Object> parameters)
+            throws Exception {
+            return null;
+        }
+    }
+
+    @Test
+    public void shouldConfigureBindingMode() throws Exception {
+        final CamelContext context = new DefaultCamelContext();
+        context.addComponent("mock-rest", new MockRest());
+
+        final RestComponent restComponent = new RestComponent();
+        restComponent.setCamelContext(context);
+
+        final RestEndpoint restEndpoint = new RestEndpoint("rest:GET:/path", restComponent);
+        restEndpoint.setComponentName("mock-rest");
+        restEndpoint.setParameters(new HashMap<>());
+
+        restEndpoint.setBindingMode(RestBindingMode.json);
+
+        final RestProducer producer = (RestProducer) restEndpoint.createProducer();
+
+        Assert.assertEquals(producer.getBindingMode(), RestBindingMode.json);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/994bea0c/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java b/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java
new file mode 100644
index 0000000..e918d31
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/component/rest/RestProducerBindingProcessorTest.java
@@ -0,0 +1,171 @@
+/**
+ * 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.component.rest;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.AsyncProcessor;
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.Message;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.apache.camel.impl.DefaultExchange;
+import org.apache.camel.impl.DefaultMessage;
+import org.apache.camel.spi.DataFormat;
+import org.easymock.Capture;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.mock;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.same;
+import static org.easymock.EasyMock.verify;
+import static org.junit.Assert.assertTrue;
+
+public class RestProducerBindingProcessorTest {
+
+    public static class RequestPojo {
+    }
+
+    public static class ResponsePojo {
+    }
+
+    final AsyncCallback callback = mock(AsyncCallback.class);
+
+    final CamelContext context = new DefaultCamelContext();
+    final DataFormat jsonDataFormat = mock(DataFormat.class);
+    final DataFormat outJsonDataFormat = mock(DataFormat.class);
+    final DataFormat outXmlDataFormat = mock(DataFormat.class);
+    final AsyncProcessor processor = mock(AsyncProcessor.class);
+    final DataFormat xmlDataFormat = mock(DataFormat.class);
+
+    @Test
+    public void shouldMarshalAndUnmarshalJson() throws Exception {
+        final String outType = ResponsePojo.class.getName();
+
+        final RestProducerBindingProcessor bindingProcessor = new RestProducerBindingProcessor(processor, context,
+            jsonDataFormat, xmlDataFormat, outJsonDataFormat, outXmlDataFormat, "json", true, outType);
+
+        final Exchange exchange = new DefaultExchange(context);
+        final Message input = new DefaultMessage();
+
+        final RequestPojo request = new RequestPojo();
+        input.setBody(request);
+        exchange.setIn(input);
+
+        jsonDataFormat.marshal(same(exchange), same(request), anyObject(OutputStream.class));
+        expectLastCall().andVoid();
+
+        final ResponsePojo response = new ResponsePojo();
+        expect(outJsonDataFormat.unmarshal(same(exchange), anyObject(InputStream.class))).andReturn(response);
+
+        final Capture<AsyncCallback> bindingCallback = EasyMock.newCapture();
+
+        expect(processor.process(same(exchange), capture(bindingCallback))).andReturn(false);
+
+        replay(jsonDataFormat, outJsonDataFormat, processor);
+
+        bindingProcessor.process(exchange, callback);
+
+        assertTrue(bindingCallback.hasCaptured());
+
+        final AsyncCallback that = bindingCallback.getValue();
+
+        that.done(false);
+
+        verify(jsonDataFormat, outJsonDataFormat, processor);
+
+        Assert.assertSame(response, exchange.getOut().getBody());
+    }
+
+    @Test
+    public void shouldMarshalAndUnmarshalXml() throws Exception {
+        final String outType = ResponsePojo.class.getName();
+
+        final RestProducerBindingProcessor bindingProcessor = new RestProducerBindingProcessor(processor, context,
+            jsonDataFormat, xmlDataFormat, outJsonDataFormat, outXmlDataFormat, "xml", true, outType);
+
+        final Exchange exchange = new DefaultExchange(context);
+        final Message input = new DefaultMessage();
+
+        final RequestPojo request = new RequestPojo();
+        input.setBody(request);
+        exchange.setIn(input);
+
+        xmlDataFormat.marshal(same(exchange), same(request), anyObject(OutputStream.class));
+        expectLastCall().andVoid();
+
+        final ResponsePojo response = new ResponsePojo();
+        expect(outXmlDataFormat.unmarshal(same(exchange), anyObject(InputStream.class))).andReturn(response);
+
+        final Capture<AsyncCallback> bindingCallback = EasyMock.newCapture();
+
+        expect(processor.process(same(exchange), capture(bindingCallback))).andReturn(false);
+
+        replay(xmlDataFormat, outXmlDataFormat, processor);
+
+        bindingProcessor.process(exchange, callback);
+
+        assertTrue(bindingCallback.hasCaptured());
+
+        final AsyncCallback that = bindingCallback.getValue();
+
+        that.done(false);
+
+        verify(xmlDataFormat, outXmlDataFormat, processor);
+
+        Assert.assertSame(response, exchange.getOut().getBody());
+    }
+
+    @Test
+    public void shouldNotMarshalAndUnmarshalByDefault() throws Exception {
+        final String outType = ResponsePojo.class.getName();
+
+        final RestProducerBindingProcessor bindingProcessor = new RestProducerBindingProcessor(processor, context,
+            jsonDataFormat, xmlDataFormat, outJsonDataFormat, outXmlDataFormat, "off", true, outType);
+
+        final Exchange exchange = new DefaultExchange(context);
+        final Message input = new DefaultMessage();
+
+        final RequestPojo request = new RequestPojo();
+        input.setBody(request);
+        exchange.setIn(input);
+
+        final Capture<AsyncCallback> bindingCallback = EasyMock.newCapture();
+
+        expect(processor.process(same(exchange), capture(bindingCallback))).andReturn(false);
+
+        replay(processor);
+
+        bindingProcessor.process(exchange, callback);
+
+        assertTrue(bindingCallback.hasCaptured());
+
+        final AsyncCallback that = bindingCallback.getValue();
+
+        that.done(false);
+
+        verify(processor);
+    }
+}