You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by nf...@apache.org on 2017/01/27 14:35:00 UTC

[1/6] camel git commit: CAMEL-10650: using registry then factory to create the service (and tests)

Repository: camel
Updated Branches:
  refs/heads/master 353eeefb6 -> 90846b586


CAMEL-10650: using registry then factory to create the service (and tests)


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

Branch: refs/heads/master
Commit: 90846b586c5160ee098f9c292d0ad1a655fe4d2a
Parents: 9cbda1a
Author: Nicola Ferraro <ni...@gmail.com>
Authored: Fri Jan 27 13:13:01 2017 +0100
Committer: Nicola Ferraro <ni...@gmail.com>
Committed: Fri Jan 27 15:31:31 2017 +0100

----------------------------------------------------------------------
 .../streams/ReactiveStreamsConsumer.java        |   4 +-
 .../streams/ReactiveStreamsEndpoint.java        |  13 +++
 .../streams/ReactiveStreamsProducer.java        |   7 +-
 .../streams/api/CamelReactiveStreams.java       |  78 +++++++++++---
 .../api/CamelReactiveStreamsService.java        |   7 --
 .../engine/CamelReactiveStreamsServiceImpl.java |   4 -
 .../camel/reactive-streams/default-service      |  18 ++++
 .../streams/CamelReactiveStreamsTest.java       |  94 +++++++++++++++++
 .../support/ReactiveStreamsTestService.java     | 103 +++++++++++++++++++
 .../apache/camel/reactive-streams/test-service  |  18 ++++
 10 files changed, 315 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
index 73d807f..081bbf4 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
@@ -51,13 +51,13 @@ public class ReactiveStreamsConsumer extends DefaultConsumer {
             executor = getEndpoint().getCamelContext().getExecutorServiceManager().newFixedThreadPool(this, getEndpoint().getEndpointUri(), poolSize);
         }
 
-        CamelReactiveStreams.get(endpoint.getCamelContext()).attachConsumer(endpoint.getStream(), this);
+        CamelReactiveStreams.get(endpoint.getCamelContext(), endpoint.getServiceName()).attachConsumer(endpoint.getStream(), this);
     }
 
     @Override
     protected void doStop() throws Exception {
         super.doStop();
-        CamelReactiveStreams.get(endpoint.getCamelContext()).detachConsumer(endpoint.getStream());
+        CamelReactiveStreams.get(endpoint.getCamelContext(), endpoint.getServiceName()).detachConsumer(endpoint.getStream());
 
         if (executor != null) {
             endpoint.getCamelContext().getExecutorServiceManager().shutdownNow(executor);

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
index 0e72c52..07a74e3 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
@@ -40,6 +40,9 @@ public class ReactiveStreamsEndpoint extends DefaultEndpoint {
     @UriParam(label = "consumer", defaultValue = "1")
     private int concurrentConsumers = 1;
 
+    @UriParam
+    private String serviceName;
+
     public ReactiveStreamsEndpoint(String endpointUri, ReactiveStreamsComponent component) {
         super(endpointUri, component);
     }
@@ -94,4 +97,14 @@ public class ReactiveStreamsEndpoint extends DefaultEndpoint {
         this.concurrentConsumers = concurrentConsumers;
     }
 
+    public String getServiceName() {
+        return serviceName;
+    }
+
+    /**
+     * Allows using an alternative CamelReactiveStreamService implementation. The implementation is looked up from the registry.
+     */
+    public void setServiceName(String serviceName) {
+        this.serviceName = serviceName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
index 3d90179..824f18d 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
@@ -25,7 +25,9 @@ import org.apache.camel.impl.DefaultAsyncProducer;
 /**
  * The Camel reactive-streams producer.
  */
-public class ReactiveStreamsProducer<T> extends DefaultAsyncProducer {
+public class ReactiveStreamsProducer extends DefaultAsyncProducer {
+
+    private ReactiveStreamsEndpoint endpoint;
 
     private String name;
 
@@ -33,6 +35,7 @@ public class ReactiveStreamsProducer<T> extends DefaultAsyncProducer {
 
     public ReactiveStreamsProducer(ReactiveStreamsEndpoint endpoint, String name) {
         super(endpoint);
+        this.endpoint = endpoint;
         this.name = name;
     }
 
@@ -51,7 +54,7 @@ public class ReactiveStreamsProducer<T> extends DefaultAsyncProducer {
     @Override
     protected void doStart() throws Exception {
         super.doStart();
-        this.service = CamelReactiveStreams.get(getEndpoint().getCamelContext());
+        this.service = CamelReactiveStreams.get(endpoint.getCamelContext(), endpoint.getServiceName());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
index 01bf23d..506ab10 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
@@ -16,8 +16,12 @@
  */
 package org.apache.camel.component.reactive.streams.api;
 
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.camel.CamelContext;
-import org.apache.camel.component.reactive.streams.engine.CamelReactiveStreamsServiceImpl;
+import org.apache.camel.spi.FactoryFinder;
 import org.apache.camel.util.ObjectHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -32,6 +36,8 @@ public final class CamelReactiveStreams {
 
     private static final Logger LOG = LoggerFactory.getLogger(CamelReactiveStreams.class);
 
+    private static Map<CamelContext, String> serviceNames = new ConcurrentHashMap<>();
+
     private CamelReactiveStreams() {
     }
 
@@ -40,37 +46,77 @@ public final class CamelReactiveStreams {
     }
 
     public static CamelReactiveStreamsService get(CamelContext context, String serviceName) {
-        CamelReactiveStreamsService service = context.hasService(CamelReactiveStreamsService.class);
-        if (service == null) {
-            service = resolveReactiveStreamsService(context, serviceName);
-            try {
-                context.addService(service, true, true);
-            } catch (Exception ex) {
-                throw new IllegalStateException("Cannot add the CamelReactiveStreamsService to the Camel context", ex);
-            }
+        if (serviceName != null && serviceName.trim().length() == 0) {
+            throw new IllegalArgumentException("the service name cannot be an empty String");
         }
 
-        if (!ObjectHelper.equal(service.getName(), serviceName)) {
+        String lookupName = serviceName != null ? serviceName : "";
+        serviceNames.computeIfAbsent(context, ctx -> {
+            CamelReactiveStreamsService service = context.hasService(CamelReactiveStreamsService.class);
+            if (service == null) {
+                service = resolveReactiveStreamsService(context, serviceName);
+                try {
+                    context.addService(service, true, true);
+                } catch (Exception ex) {
+                    throw new IllegalStateException("Cannot add the CamelReactiveStreamsService to the Camel context", ex);
+                }
+            }
+
+            return lookupName;
+        });
+
+        if (!ObjectHelper.equal(serviceNames.get(context), lookupName)) {
             // only a single implementation of the CamelReactiveStreamService can be present per Camel context
             throw new IllegalArgumentException("Cannot use two different implementations of CamelReactiveStreamsService in the same CamelContext: "
-                    + "existing service name [" + service.getName() + "] - requested [" + serviceName + "]");
+                    + "existing service name [" + serviceNames.get(context) + "] - requested [" + lookupName + "]");
         }
 
-        return service;
+        return context.hasService(CamelReactiveStreamsService.class);
     }
 
     private static CamelReactiveStreamsService resolveReactiveStreamsService(CamelContext context, String serviceName) {
         CamelReactiveStreamsService service = null;
         if (serviceName != null) {
+            // lookup in the registry
             service = context.getRegistry().lookupByNameAndType(serviceName, CamelReactiveStreamsService.class);
-        }
 
-        if (service == null) {
-            LOG.info("Using default reactive stream service");
-            service = new CamelReactiveStreamsServiceImpl();
+            if (service == null) {
+                service = resolveServiceUsingFactory(context, serviceName);
+            }
+        } else {
+            Set<CamelReactiveStreamsService> set = context.getRegistry().findByType(CamelReactiveStreamsService.class);
+            if (set.size() == 1) {
+                service = set.iterator().next();
+            }
+
+            if (service == null) {
+                LOG.info("Using default reactive stream service");
+                service = resolveServiceUsingFactory(context, null);
+            }
         }
 
         return service;
     }
 
+    @SuppressWarnings("unchecked")
+    private static CamelReactiveStreamsService resolveServiceUsingFactory(CamelContext context, String name) {
+        if (name == null) {
+            name = "default-service";
+        }
+
+        String path = "META-INF/services/org/apache/camel/reactive-streams/";
+        Class<? extends CamelReactiveStreamsService> serviceClass = null;
+        try {
+            FactoryFinder finder = context.getFactoryFinder(path);
+            LOG.trace("Using FactoryFinder: {}", finder);
+            serviceClass = (Class<? extends CamelReactiveStreamsService>) finder.findClass(name);
+            return serviceClass.newInstance();
+        } catch (ClassNotFoundException e) {
+            throw new IllegalStateException("Class referenced in '" + path + name + "' not found", e);
+        } catch (Exception e) {
+            throw new IllegalStateException("Unable to create the reactive stream service defined in '" + path + name + "'", e);
+        }
+
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
index a397ce9..57c635e 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
@@ -111,11 +111,4 @@ public interface CamelReactiveStreamsService extends CamelContextAware, Service
      */
     void detachConsumer(String name);
 
-    /**
-     * Returns the name of this service implementation if present.
-     * The name of any named implementation must match their lookup key in the registry.
-     * @return the name of the service implementation or null (for the default implementation)
-     */
-    String getName();
-
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
index ac171de..ef617be 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
@@ -131,8 +131,4 @@ public class CamelReactiveStreamsServiceImpl implements CamelReactiveStreamsServ
         return this.context;
     }
 
-    @Override
-    public String getName() {
-        return null;
-    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/reactive-streams/default-service
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/reactive-streams/default-service b/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/reactive-streams/default-service
new file mode 100644
index 0000000..2ce7448
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/reactive-streams/default-service
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+class=org.apache.camel.component.reactive.streams.engine.CamelReactiveStreamsServiceImpl

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelReactiveStreamsTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelReactiveStreamsTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelReactiveStreamsTest.java
new file mode 100644
index 0000000..149e317
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelReactiveStreamsTest.java
@@ -0,0 +1,94 @@
+/**
+ * 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.reactive.streams;
+
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreamsService;
+import org.apache.camel.component.reactive.streams.engine.CamelReactiveStreamsServiceImpl;
+import org.apache.camel.component.reactive.streams.support.ReactiveStreamsTestService;
+import org.apache.camel.impl.JndiRegistry;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+
+public class CamelReactiveStreamsTest extends CamelTestSupport {
+
+    @Test
+    public void testDefaultService() {
+        CamelReactiveStreamsService service1 = CamelReactiveStreams.get(context, "default-service");
+        assertTrue(service1 instanceof CamelReactiveStreamsServiceImpl);
+    }
+
+    @Test
+    public void testSameDefaultServiceReturned() {
+        CamelReactiveStreamsService service1 = CamelReactiveStreams.get(context, "default-service");
+        CamelReactiveStreamsService service2 = CamelReactiveStreams.get(context, "default-service");
+        assertTrue(service1 instanceof CamelReactiveStreamsServiceImpl);
+        assertEquals(service1, service2);
+    }
+
+    @Test
+    public void testSameServiceReturnedFromRegistry() {
+        CamelReactiveStreamsService service1 = CamelReactiveStreams.get(context);
+        CamelReactiveStreamsService service2 = CamelReactiveStreams.get(context);
+
+        assertEquals(service1, service2);
+        assertTrue(service1 instanceof ReactiveStreamsTestService);
+        assertEquals("from-registry", ((ReactiveStreamsTestService) service1).getName());
+    }
+
+    @Test
+    public void testSameNamedServiceReturnedFromRegistry() {
+        CamelReactiveStreamsService service1 = CamelReactiveStreams.get(context, "dummy");
+        CamelReactiveStreamsService service2 = CamelReactiveStreams.get(context, "dummy");
+
+        assertEquals(service1, service2);
+        assertTrue(service1 instanceof ReactiveStreamsTestService);
+        assertEquals("from-registry", ((ReactiveStreamsTestService) service1).getName());
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testOnlyOneService() {
+        CamelReactiveStreams.get(context);
+        CamelReactiveStreams.get(context, "dummy");
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testOnlyOneNamedService() {
+        CamelReactiveStreams.get(context, "dummy");
+        CamelReactiveStreams.get(context, "dummy2");
+    }
+
+    @Test
+    public void testNamedServiceResolvedUsingFactory() {
+        CamelReactiveStreamsService service1 = CamelReactiveStreams.get(context, "test-service");
+        assertTrue(service1 instanceof ReactiveStreamsTestService);
+        assertNull(((ReactiveStreamsTestService) service1).getName());
+    }
+
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        JndiRegistry registry = super.createRegistry();
+        registry.bind("dummy", new ReactiveStreamsTestService("from-registry"));
+        return registry;
+    }
+
+    @Override
+    public boolean isUseRouteBuilder() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/ReactiveStreamsTestService.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/ReactiveStreamsTestService.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/ReactiveStreamsTestService.java
new file mode 100644
index 0000000..b46c9ff
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/ReactiveStreamsTestService.java
@@ -0,0 +1,103 @@
+/**
+ * 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.reactive.streams.support;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsConsumer;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreamsService;
+import org.apache.camel.component.reactive.streams.api.DispatchCallback;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+
+/**
+ * Test (dummy) service for reactive streams.
+ */
+public class ReactiveStreamsTestService implements CamelReactiveStreamsService {
+
+    private String name;
+
+    public ReactiveStreamsTestService() {
+    }
+
+    public ReactiveStreamsTestService(String name) {
+        this.name = name;
+    }
+
+    @Override
+    public void start() throws Exception {
+
+    }
+
+    @Override
+    public void setCamelContext(CamelContext camelContext) {
+
+    }
+
+    @Override
+    public void stop() throws Exception {
+
+    }
+
+    @Override
+    public CamelContext getCamelContext() {
+        return null;
+    }
+
+    @Override
+    public Publisher<Exchange> getPublisher(String name) {
+        return null;
+    }
+
+    @Override
+    public <T> Publisher<T> getPublisher(String name, Class<T> type) {
+        return null;
+    }
+
+    @Override
+    public Subscriber<Exchange> getSubscriber(String name) {
+        return null;
+    }
+
+    @Override
+    public <T> Subscriber<T> getSubscriber(String name, Class<T> type) {
+        return null;
+    }
+
+    @Override
+    public void process(String name, Exchange exchange, DispatchCallback<Exchange> callback) {
+
+    }
+
+    @Override
+    public void attachConsumer(String name, ReactiveStreamsConsumer consumer) {
+
+    }
+
+    @Override
+    public void detachConsumer(String name) {
+
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/90846b58/components/camel-reactive-streams/src/test/resources/META-INF/services/org/apache/camel/reactive-streams/test-service
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/resources/META-INF/services/org/apache/camel/reactive-streams/test-service b/components/camel-reactive-streams/src/test/resources/META-INF/services/org/apache/camel/reactive-streams/test-service
new file mode 100644
index 0000000..03f7827
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/resources/META-INF/services/org/apache/camel/reactive-streams/test-service
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+class=org.apache.camel.component.reactive.streams.support.ReactiveStreamsTestService


[3/6] camel git commit: CAMEL-10650: adding lookup into registry for named services

Posted by nf...@apache.org.
CAMEL-10650: adding lookup into registry for named services

CAMEL-10650: adding lookup into registry for named services


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

Branch: refs/heads/master
Commit: 9cbda1a1042e0ce463fa4a2b57ca56369137b934
Parents: 40b42e6
Author: Nicola Ferraro <ni...@gmail.com>
Authored: Fri Jan 27 10:48:59 2017 +0100
Committer: Nicola Ferraro <ni...@gmail.com>
Committed: Fri Jan 27 15:31:31 2017 +0100

----------------------------------------------------------------------
 .../streams/ReactiveStreamsConsumer.java        |   4 +-
 .../streams/ReactiveStreamsProducer.java        |   2 +-
 .../streams/api/CamelReactiveStreams.java       | 136 ++++---------------
 .../api/CamelReactiveStreamsService.java        |  13 +-
 .../reactive/streams/api/DispatchCallback.java  |   1 +
 .../engine/CamelReactiveStreamsServiceImpl.java |   5 +
 components/readme.adoc                          |   2 +-
 7 files changed, 43 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/9cbda1a1/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
index ca19e0a..73d807f 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
@@ -51,13 +51,13 @@ public class ReactiveStreamsConsumer extends DefaultConsumer {
             executor = getEndpoint().getCamelContext().getExecutorServiceManager().newFixedThreadPool(this, getEndpoint().getEndpointUri(), poolSize);
         }
 
-        CamelReactiveStreams.get(endpoint.getCamelContext()).getService().attachConsumer(endpoint.getStream(), this);
+        CamelReactiveStreams.get(endpoint.getCamelContext()).attachConsumer(endpoint.getStream(), this);
     }
 
     @Override
     protected void doStop() throws Exception {
         super.doStop();
-        CamelReactiveStreams.get(endpoint.getCamelContext()).getService().detachConsumer(endpoint.getStream());
+        CamelReactiveStreams.get(endpoint.getCamelContext()).detachConsumer(endpoint.getStream());
 
         if (executor != null) {
             endpoint.getCamelContext().getExecutorServiceManager().shutdownNow(executor);

http://git-wip-us.apache.org/repos/asf/camel/blob/9cbda1a1/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
index d74cdb1..3d90179 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
@@ -51,7 +51,7 @@ public class ReactiveStreamsProducer<T> extends DefaultAsyncProducer {
     @Override
     protected void doStart() throws Exception {
         super.doStart();
-        this.service = CamelReactiveStreams.get(getEndpoint().getCamelContext()).getService();
+        this.service = CamelReactiveStreams.get(getEndpoint().getCamelContext());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/9cbda1a1/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
index 1bafc5c..01bf23d 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
@@ -16,74 +16,53 @@
  */
 package org.apache.camel.component.reactive.streams.api;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.camel.CamelContext;
-import org.apache.camel.Exchange;
 import org.apache.camel.component.reactive.streams.engine.CamelReactiveStreamsServiceImpl;
-import org.apache.camel.spi.FactoryFinder;
-import org.reactivestreams.Publisher;
-import org.reactivestreams.Subscriber;
+import org.apache.camel.util.ObjectHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * This is the main entry-point for getting Camel streams associate to reactive-streams endpoints.
  *
- * It delegates main methods to an instance of {@link CamelReactiveStreamsService}. This component provides
- * a default implementation that can be overridden in a 'META-INF/services/reactive-streams/reactiveStreamsService' file.
+ * It allows to retrieve the {@link CamelReactiveStreamsService} to access Camel streams.
+ * This class returns the default implementation of the service unless the client requests a named service,
  */
 public final class CamelReactiveStreams {
 
     private static final Logger LOG = LoggerFactory.getLogger(CamelReactiveStreams.class);
 
-    private static Map<CamelContext, CamelReactiveStreams> instances = new ConcurrentHashMap<>();
-
-    private CamelReactiveStreamsService service;
+    private CamelReactiveStreams() {
+    }
 
-    private CamelReactiveStreams(CamelReactiveStreamsService service) {
-        this.service = service;
+    public static CamelReactiveStreamsService get(CamelContext context) {
+        return get(context, null);
     }
 
-    public static CamelReactiveStreams get(CamelContext context) {
-        instances.computeIfAbsent(context, ctx -> {
-            CamelReactiveStreamsService service = resolveReactiveStreamsService(context);
+    public static CamelReactiveStreamsService get(CamelContext context, String serviceName) {
+        CamelReactiveStreamsService service = context.hasService(CamelReactiveStreamsService.class);
+        if (service == null) {
+            service = resolveReactiveStreamsService(context, serviceName);
             try {
-                ctx.addService(service, true, true);
+                context.addService(service, true, true);
             } catch (Exception ex) {
                 throw new IllegalStateException("Cannot add the CamelReactiveStreamsService to the Camel context", ex);
             }
-            return new CamelReactiveStreams(service);
-        });
-
-        return instances.get(context);
-    }
+        }
 
-    @SuppressWarnings("unchecked")
-    private static CamelReactiveStreamsService resolveReactiveStreamsService(CamelContext context) {
-        Class<? extends CamelReactiveStreamsService> serviceClass = null;
-        try {
-            FactoryFinder finder = context.getFactoryFinder("META-INF/services/reactive-streams/");
-            LOG.trace("Using FactoryFinder: {}", finder);
-            serviceClass = (Class<? extends CamelReactiveStreamsService>) finder.findClass("reactiveStreamsService");
-        } catch (ClassNotFoundException e) {
-            LOG.trace("'reactive.streams.service.class' not found", e);
-        } catch (IOException e) {
-            LOG.trace("No reactive stream service defined in 'META-INF/services/org/apache/camel/component/'", e);
+        if (!ObjectHelper.equal(service.getName(), serviceName)) {
+            // only a single implementation of the CamelReactiveStreamService can be present per Camel context
+            throw new IllegalArgumentException("Cannot use two different implementations of CamelReactiveStreamsService in the same CamelContext: "
+                    + "existing service name [" + service.getName() + "] - requested [" + serviceName + "]");
         }
 
+        return service;
+    }
+
+    private static CamelReactiveStreamsService resolveReactiveStreamsService(CamelContext context, String serviceName) {
         CamelReactiveStreamsService service = null;
-        if (serviceClass != null) {
-            try {
-                service = serviceClass.newInstance();
-                LOG.info("Created reactive stream service from class: " + serviceClass.getName());
-            } catch (Exception e) {
-                LOG.debug("Unable to create a reactive stream service of class " + serviceClass.getName(), e);
-            }
+        if (serviceName != null) {
+            service = context.getRegistry().lookupByNameAndType(serviceName, CamelReactiveStreamsService.class);
         }
 
         if (service == null) {
@@ -94,73 +73,4 @@ public final class CamelReactiveStreams {
         return service;
     }
 
-    /**
-     * Allows retrieving the service responsible for binding camel routes to streams.
-     *
-     * @return the stream service
-     */
-    public CamelReactiveStreamsService getService() {
-        return service;
-    }
-
-    /**
-     * Returns the publisher associated to the given stream name.
-     * A publisher can be used to push Camel exchanges to reactive-streams subscribers.
-     *
-     * @param name the stream name
-     * @return the stream publisher
-     */
-    public Publisher<Exchange> getPublisher(String name) {
-        Objects.requireNonNull(name, "name cannot be null");
-
-        return service.getPublisher(name);
-    }
-
-    /**
-     * Returns the publisher associated to the given stream name.
-     * A publisher can be used to push Camel exchange to external reactive-streams subscribers.
-     *
-     * The publisher converts automatically exchanges to the given type.
-     *
-     * @param name the stream name
-     * @param type the type of the emitted items
-     * @param <T> the type of items emitted by the publisher
-     * @return the publisher associated to the stream
-     */
-    public <T> Publisher<T> getPublisher(String name, Class<T> type) {
-        Objects.requireNonNull(name, "name cannot be null");
-
-        return service.getPublisher(name, type);
-    }
-
-    /**
-     * Returns the subscriber associated to the given stream name.
-     * A subscriber can be used to push items coming from external reactive-streams publishers to Camel routes.
-     *
-     * @param name the stream name
-     * @return the subscriber associated with the stream
-     */
-    public Subscriber<Exchange> getSubscriber(String name) {
-        Objects.requireNonNull(name, "name cannot be null");
-
-        return service.getSubscriber(name);
-    }
-
-    /**
-     * Returns the subscriber associated to the given stream name.
-     * A subscriber can be used to push items coming from external reactive-streams publishers to Camel routes.
-     *
-     * The subscriber converts automatically items of the given type to exchanges before pushing them.
-     *
-     * @param name the stream name
-     * @param type the publisher converts automatically exchanges to the given type.
-     * @param <T> the type of items accepted by the subscriber
-     * @return the subscriber associated with the stream
-     */
-    public <T> Subscriber<T> getSubscriber(String name, Class<T> type) {
-        Objects.requireNonNull(name, "name cannot be null");
-
-        return service.getSubscriber(name, type);
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/9cbda1a1/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
index 147ece7..a397ce9 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
@@ -81,7 +81,7 @@ public interface CamelReactiveStreamsService extends CamelContextAware, Service
      */
 
     /**
-     * Sends the exchange to all active subscriptions on the given stream.
+     * Used by Camel to send the exchange to all active subscriptions on the given stream.
      * The callback is used to signal that the exchange has been delivered to the subscribers.
      *
      * @param name the stream name
@@ -95,7 +95,7 @@ public interface CamelReactiveStreamsService extends CamelContextAware, Service
      */
 
     /**
-     * Associate the subscriber of the stream with the given name to a specific Camel consumer.
+     * Used by Camel to associate the subscriber of the stream with the given name to a specific Camel consumer.
      * This method is used to bind a Camel route to a reactive stream.
      *
      * @param name the stream name
@@ -105,10 +105,17 @@ public interface CamelReactiveStreamsService extends CamelContextAware, Service
     void attachConsumer(String name, ReactiveStreamsConsumer consumer);
 
     /**
-     * Deassociate the existing consumer from the given stream.
+     * Used by Camel to detach the existing consumer from the given stream.
      *
      * @param name the stream name
      */
     void detachConsumer(String name);
 
+    /**
+     * Returns the name of this service implementation if present.
+     * The name of any named implementation must match their lookup key in the registry.
+     * @return the name of the service implementation or null (for the default implementation)
+     */
+    String getName();
+
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/9cbda1a1/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java
index c07f3d3..01785d4 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java
@@ -19,6 +19,7 @@ package org.apache.camel.component.reactive.streams.api;
 /**
  * A callback used to signal when a item coming from a Camel route has been delivered to the external stream processor.
  */
+@FunctionalInterface
 public interface DispatchCallback<T> {
 
     /**

http://git-wip-us.apache.org/repos/asf/camel/blob/9cbda1a1/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
index fb6b693..ac171de 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
@@ -130,4 +130,9 @@ public class CamelReactiveStreamsServiceImpl implements CamelReactiveStreamsServ
     public CamelContext getCamelContext() {
         return this.context;
     }
+
+    @Override
+    public String getName() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/9cbda1a1/components/readme.adoc
----------------------------------------------------------------------
diff --git a/components/readme.adoc b/components/readme.adoc
index b8fb246..190f8ea 100644
--- a/components/readme.adoc
+++ b/components/readme.adoc
@@ -439,7 +439,7 @@ Components
 `rabbitmq:hostname:portNumber/exchangeName` | The rabbitmq component allows you produce and consume messages from RabbitMQ instances.
 
 | link:camel-reactive-streams/src/main/docs/reactive-streams-component.adoc[Reactive Streams] (camel-reactive-streams) +
-`reactive-streams:/stream` | The Camel reactive-streams endpoint.
+`reactive-streams:stream` | The Camel reactive-streams endpoint.
 
 | link:camel-restlet/src/main/docs/restlet-component.adoc[Restlet] (camel-restlet) +
 `restlet:protocol:host:port/uriPattern` | Component for consuming and producing Restful resources using Restlet.


[4/6] camel git commit: CAMEL-10612: Reactive streams

Posted by nf...@apache.org.
http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentConfiguration.java
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentConfiguration.java b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentConfiguration.java
new file mode 100644
index 0000000..a934d79
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentConfiguration.java
@@ -0,0 +1,100 @@
+/**
+ * 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.reactive.streams.springboot;
+
+import org.apache.camel.component.reactive.streams.ReactiveStreamsBackpressureStrategy;
+import org.springframework.boot.context.properties.ConfigurationProperties;
+
+/**
+ * The Camel reactive-streams endpoint.
+ * 
+ * Generated by camel-package-maven-plugin - do not edit this file!
+ */
+@ConfigurationProperties(prefix = "camel.component.reactive-streams")
+public class ReactiveStreamsComponentConfiguration {
+
+    /**
+     * Configures the internal engine for Reactive Streams.
+     */
+    private ReactiveStreamsEngineConfigurationNestedConfiguration internalEngineConfiguration;
+    /**
+     * The backpressure strategy to use when pushing events to a slow
+     * subscriber.
+     */
+    private ReactiveStreamsBackpressureStrategy backpressureStrategy;
+
+    public ReactiveStreamsEngineConfigurationNestedConfiguration getInternalEngineConfiguration() {
+        return internalEngineConfiguration;
+    }
+
+    public void setInternalEngineConfiguration(
+            ReactiveStreamsEngineConfigurationNestedConfiguration internalEngineConfiguration) {
+        this.internalEngineConfiguration = internalEngineConfiguration;
+    }
+
+    public ReactiveStreamsBackpressureStrategy getBackpressureStrategy() {
+        return backpressureStrategy;
+    }
+
+    public void setBackpressureStrategy(
+            ReactiveStreamsBackpressureStrategy backpressureStrategy) {
+        this.backpressureStrategy = backpressureStrategy;
+    }
+
+    public static class ReactiveStreamsEngineConfigurationNestedConfiguration {
+        public static final Class CAMEL_NESTED_CLASS = org.apache.camel.component.reactive.streams.engine.ReactiveStreamsEngineConfiguration.class;
+        /**
+         * The name of the thread pool used by the reactive streams internal
+         * engine.
+         */
+        private String threadPoolName;
+        /**
+         * The minimum number of threads used by the reactive streams internal
+         * engine.
+         */
+        private Integer threadPoolMinSize;
+        /**
+         * The maximum number of threads used by the reactive streams internal
+         * engine.
+         */
+        private Integer threadPoolMaxSize;
+
+        public String getThreadPoolName() {
+            return threadPoolName;
+        }
+
+        public void setThreadPoolName(String threadPoolName) {
+            this.threadPoolName = threadPoolName;
+        }
+
+        public Integer getThreadPoolMinSize() {
+            return threadPoolMinSize;
+        }
+
+        public void setThreadPoolMinSize(Integer threadPoolMinSize) {
+            this.threadPoolMinSize = threadPoolMinSize;
+        }
+
+        public Integer getThreadPoolMaxSize() {
+            return threadPoolMaxSize;
+        }
+
+        public void setThreadPoolMaxSize(Integer threadPoolMaxSize) {
+            this.threadPoolMaxSize = threadPoolMaxSize;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/LICENSE.txt
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/LICENSE.txt b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/LICENSE.txt
new file mode 100644
index 0000000..6b0b127
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/LICENSE.txt
@@ -0,0 +1,203 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/NOTICE.txt
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/NOTICE.txt b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/NOTICE.txt
new file mode 100644
index 0000000..2e215bf
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/NOTICE.txt
@@ -0,0 +1,11 @@
+   =========================================================================
+   ==  NOTICE file corresponding to the section 4 d of                    ==
+   ==  the Apache License, Version 2.0,                                   ==
+   ==  in this case for the Apache Camel distribution.                    ==
+   =========================================================================
+
+   This product includes software developed by
+   The Apache Software Foundation (http://www.apache.org/).
+
+   Please read the different LICENSE files present in the licenses directory of
+   this distribution.

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/additional-spring-configuration-metadata.json
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/additional-spring-configuration-metadata.json b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/additional-spring-configuration-metadata.json
new file mode 100644
index 0000000..c11a2aa
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/additional-spring-configuration-metadata.json
@@ -0,0 +1,10 @@
+{
+  "properties": [
+    {
+      "defaultValue": true,
+      "name": "camel.component.reactive-streams.enabled",
+      "description": "Enable reactive-streams component",
+      "type": "java.lang.Boolean"
+    }
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.factories
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.factories b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.factories
new file mode 100644
index 0000000..4a5e152
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.factories
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+org.springframework.boot.autoconfigure.EnableAutoConfiguration=\
+org.apache.camel.component.reactive.streams.springboot.ReactiveStreamsComponentAutoConfiguration

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.provides
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.provides b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.provides
new file mode 100644
index 0000000..8f5e88d
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/resources/META-INF/spring.provides
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+provides: camel-reactive-streams
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/pom.xml
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/pom.xml b/platforms/spring-boot/components-starter/pom.xml
index 8931b12..b49dbf2 100644
--- a/platforms/spring-boot/components-starter/pom.xml
+++ b/platforms/spring-boot/components-starter/pom.xml
@@ -232,6 +232,7 @@
     <module>camel-quartz2-starter</module>
     <module>camel-quickfix-starter</module>
     <module>camel-rabbitmq-starter</module>
+    <module>camel-reactive-streams-starter</module>
     <module>camel-restlet-starter</module>
     <module>camel-ribbon-starter</module>
     <module>camel-rmi-starter</module>


[5/6] camel git commit: CAMEL-10612: Reactive streams

Posted by nf...@apache.org.
http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/resources/META-INF/LICENSE.txt
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/resources/META-INF/LICENSE.txt b/components/camel-reactive-streams/src/main/resources/META-INF/LICENSE.txt
new file mode 100644
index 0000000..6b0b127
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/resources/META-INF/LICENSE.txt
@@ -0,0 +1,203 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/resources/META-INF/NOTICE.txt
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/resources/META-INF/NOTICE.txt b/components/camel-reactive-streams/src/main/resources/META-INF/NOTICE.txt
new file mode 100644
index 0000000..2e215bf
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/resources/META-INF/NOTICE.txt
@@ -0,0 +1,11 @@
+   =========================================================================
+   ==  NOTICE file corresponding to the section 4 d of                    ==
+   ==  the Apache License, Version 2.0,                                   ==
+   ==  in this case for the Apache Camel distribution.                    ==
+   =========================================================================
+
+   This product includes software developed by
+   The Apache Software Foundation (http://www.apache.org/).
+
+   Please read the different LICENSE files present in the licenses directory of
+   this distribution.

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/component/reactive-streams
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/component/reactive-streams b/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/component/reactive-streams
new file mode 100644
index 0000000..3ea3f1c
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/resources/META-INF/services/org/apache/camel/component/reactive-streams
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+class=org.apache.camel.component.reactive.streams.ReactiveStreamsComponent

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressurePublisherRoutePolicyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressurePublisherRoutePolicyTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressurePublisherRoutePolicyTest.java
new file mode 100644
index 0000000..81d03bc
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressurePublisherRoutePolicyTest.java
@@ -0,0 +1,165 @@
+/**
+ * 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.reactive.streams;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import io.reactivex.Observable;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.LoggingLevel;
+import org.apache.camel.StatefulService;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.component.reactive.streams.support.TestSubscriber;
+import org.apache.camel.impl.ThrottlingInflightRoutePolicy;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+import org.reactivestreams.Publisher;
+
+public class BackpressurePublisherRoutePolicyTest extends CamelTestSupport {
+
+    @Test
+    public void testThatBackpressureCausesTemporaryRouteStop() throws Exception {
+
+        CountDownLatch generationLatch = new CountDownLatch(25);
+
+        new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                ThrottlingInflightRoutePolicy policy = new ThrottlingInflightRoutePolicy();
+                policy.setMaxInflightExchanges(10);
+                policy.setScope(ThrottlingInflightRoutePolicy.ThrottlingScope.Route);
+                policy.setResumePercentOfMax(70);
+                policy.setLoggingLevel(LoggingLevel.INFO);
+
+                from("timer:tick?period=50&repeatCount=35")
+                        .id("policy-route")
+                        .routePolicy(policy)
+                        .process(x -> generationLatch.countDown())
+                        .to("reactive-streams:pub");
+            }
+        }.addRoutesToCamelContext(context);
+
+        CountDownLatch receptionLatch = new CountDownLatch(35);
+
+        Publisher<Exchange> pub = CamelReactiveStreams.get(context()).getPublisher("pub", Exchange.class);
+        TestSubscriber<Exchange> subscriber = new TestSubscriber<Exchange>() {
+            @Override
+            public void onNext(Exchange o) {
+                super.onNext(o);
+                receptionLatch.countDown();
+            }
+        };
+        subscriber.setInitiallyRequested(10);
+        pub.subscribe(subscriber);
+
+        // Add another (fast) subscription that should not affect the backpressure on the route
+        Observable.fromPublisher(pub)
+                .subscribe();
+
+        context.start();
+
+        generationLatch.await(5, TimeUnit.SECONDS); // after 25 messages are generated
+        // The number of exchanges should be 10 (requested by the subscriber), so 35-10=25
+        assertEquals(25, receptionLatch.getCount());
+
+        // fire a delayed request from the subscriber (required by camel core)
+        subscriber.request(1);
+        Thread.sleep(250);
+
+        StatefulService service = (StatefulService) context().getRoute("policy-route").getConsumer();
+        // ensure the route is stopped or suspended
+        assertTrue(service.isStopped() || service.isSuspended());
+
+        // request all the remaining exchanges
+        subscriber.request(24);
+        assertTrue(receptionLatch.await(5, TimeUnit.SECONDS));
+        // The reception latch has gone to 0
+    }
+
+    @Test
+    public void testThatRouteRestartsOnUnsubscription() throws Exception {
+
+        CountDownLatch generationLatch = new CountDownLatch(25);
+
+        new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                ThrottlingInflightRoutePolicy policy = new ThrottlingInflightRoutePolicy();
+                policy.setMaxInflightExchanges(10);
+                policy.setScope(ThrottlingInflightRoutePolicy.ThrottlingScope.Route);
+                policy.setResumePercentOfMax(70);
+                policy.setLoggingLevel(LoggingLevel.INFO);
+
+                from("timer:tick?period=50") // unbounded
+                        .id("policy-route")
+                        .routePolicy(policy)
+                        .process(x -> generationLatch.countDown())
+                        .to("reactive-streams:pub");
+            }
+        }.addRoutesToCamelContext(context);
+
+        CountDownLatch receptionLatch = new CountDownLatch(35);
+
+        Publisher<Exchange> pub = CamelReactiveStreams.get(context()).getPublisher("pub", Exchange.class);
+        TestSubscriber<Exchange> subscriber = new TestSubscriber<Exchange>() {
+            @Override
+            public void onNext(Exchange o) {
+                super.onNext(o);
+                receptionLatch.countDown();
+            }
+        };
+        subscriber.setInitiallyRequested(10);
+        pub.subscribe(subscriber);
+
+        // Add another (fast) subscription that should not affect the backpressure on the route
+        Observable.fromPublisher(pub)
+                .subscribe();
+
+        context.start();
+
+        generationLatch.await(5, TimeUnit.SECONDS); // after 25 messages are generated
+        // The number of exchanges should be 10 (requested by the subscriber), so 35-10=25
+        assertEquals(25, receptionLatch.getCount());
+
+        // fire a delayed request from the subscriber (required by camel core)
+        subscriber.request(1);
+        Thread.sleep(250);
+
+        StatefulService service = (StatefulService) context().getRoute("policy-route").getConsumer();
+        // ensure the route is stopped or suspended
+        assertTrue(service.isStopped() || service.isSuspended());
+        subscriber.cancel();
+
+        // request other exchanges to ensure that the route works
+        CountDownLatch latch = new CountDownLatch(20);
+        Observable.fromPublisher(pub)
+                .subscribe(n -> {
+                    latch.countDown();
+                });
+
+        assertTrue(latch.await(5, TimeUnit.SECONDS));
+    }
+
+    @Override
+    public boolean isUseRouteBuilder() {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureStrategyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureStrategyTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureStrategyTest.java
new file mode 100644
index 0000000..8846cd1
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureStrategyTest.java
@@ -0,0 +1,164 @@
+/**
+ * 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.reactive.streams;
+
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import io.reactivex.Flowable;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.component.reactive.streams.support.TestSubscriber;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class BackpressureStrategyTest extends CamelTestSupport {
+
+    @Test
+    public void testBackpressureBufferStrategy() throws Exception {
+
+        new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:gen?period=20&repeatCount=20")
+                        .setBody().header(Exchange.TIMER_COUNTER)
+                        .to("reactive-streams:integers");
+            }
+        }.addRoutesToCamelContext(context);
+
+        Flowable<Integer> integers = Flowable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("integers", Integer.class));
+
+        ConcurrentLinkedQueue<Integer> queue = new ConcurrentLinkedQueue<>();
+        CountDownLatch latch = new CountDownLatch(1);
+
+        Flowable.interval(0, 50, TimeUnit.MILLISECONDS)
+                .zipWith(integers, (l, i) -> i)
+                .timeout(2000, TimeUnit.MILLISECONDS, Flowable.empty())
+                .doOnComplete(latch::countDown)
+                .subscribe(queue::add);
+
+        context().start();
+
+        assertTrue(latch.await(5, TimeUnit.SECONDS));
+
+        assertEquals(20, queue.size());
+        int num = 1;
+        for (int i : queue) {
+            assertEquals(num++, i);
+        }
+    }
+
+    @Test
+    public void testBackpressureDropStrategy() throws Exception {
+
+        ReactiveStreamsComponent comp = (ReactiveStreamsComponent) context().getComponent("reactive-streams");
+        comp.setBackpressureStrategy(ReactiveStreamsBackpressureStrategy.DROP);
+
+        new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:gen?period=20&repeatCount=20")
+                        .setBody().header(Exchange.TIMER_COUNTER)
+                        .to("reactive-streams:integers");
+            }
+        }.addRoutesToCamelContext(context);
+
+
+        ConcurrentLinkedQueue<Integer> queue = new ConcurrentLinkedQueue<>();
+        final CountDownLatch latch = new CountDownLatch(1);
+        final CountDownLatch latch2 = new CountDownLatch(2);
+
+        TestSubscriber<Integer> subscriber = new TestSubscriber<Integer>() {
+            @Override
+            public void onNext(Integer o) {
+                queue.add(o);
+                latch.countDown();
+                latch2.countDown();
+            }
+        };
+        subscriber.setInitiallyRequested(1);
+        CamelReactiveStreams.get(context).getPublisher("integers", Integer.class).subscribe(subscriber);
+
+        context().start();
+
+        assertTrue(latch.await(5, TimeUnit.SECONDS));
+        Thread.sleep(1000); // wait for all numbers to be generated
+
+        subscriber.request(19);
+        assertTrue(latch2.await(1, TimeUnit.SECONDS));
+        Thread.sleep(200); // add other time to ensure no other items arrive
+        assertEquals(2, queue.size());
+        int sum = queue.stream().reduce((i, j) -> i + j).get();
+        assertEquals(3, sum); // 1 + 2 = 3
+
+        subscriber.cancel();
+    }
+
+    @Test
+    public void testBackpressureLatestStrategy() throws Exception {
+
+        ReactiveStreamsComponent comp = (ReactiveStreamsComponent) context().getComponent("reactive-streams");
+        comp.setBackpressureStrategy(ReactiveStreamsBackpressureStrategy.LATEST);
+
+        new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:gen?period=20&repeatCount=20")
+                        .setBody().header(Exchange.TIMER_COUNTER)
+                        .to("reactive-streams:integers");
+            }
+        }.addRoutesToCamelContext(context);
+
+
+        ConcurrentLinkedQueue<Integer> queue = new ConcurrentLinkedQueue<>();
+        final CountDownLatch latch = new CountDownLatch(1);
+        final CountDownLatch latch2 = new CountDownLatch(2);
+
+        TestSubscriber<Integer> subscriber = new TestSubscriber<Integer>() {
+            @Override
+            public void onNext(Integer o) {
+                queue.add(o);
+                latch.countDown();
+                latch2.countDown();
+            }
+        };
+        subscriber.setInitiallyRequested(1);
+        CamelReactiveStreams.get(context).getPublisher("integers", Integer.class).subscribe(subscriber);
+
+        context().start();
+
+        assertTrue(latch.await(5, TimeUnit.SECONDS));
+        Thread.sleep(1000); // wait for all numbers to be generated
+
+        subscriber.request(19);
+        assertTrue(latch2.await(1, TimeUnit.SECONDS));
+        Thread.sleep(200); // add other time to ensure no other items arrive
+        assertEquals(2, queue.size());
+        int sum = queue.stream().reduce((i, j) -> i + j).get();
+        assertEquals(21, sum); // 1 + 20 = 21
+
+        subscriber.cancel();
+    }
+
+    @Override
+    public boolean isUseRouteBuilder() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureSubscriberTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureSubscriberTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureSubscriberTest.java
new file mode 100644
index 0000000..edcad30
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BackpressureSubscriberTest.java
@@ -0,0 +1,107 @@
+/**
+ * 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.reactive.streams;
+
+import io.reactivex.BackpressureStrategy;
+import io.reactivex.Flowable;
+import io.reactivex.Observable;
+
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+/**
+ * Test backpressure from the consumer side.
+ */
+public class BackpressureSubscriberTest extends CamelTestSupport {
+
+    @Test
+    public void testBackpressure() throws Exception {
+
+        long start = System.currentTimeMillis();
+        Observable.range(0, 10)
+                .toFlowable(BackpressureStrategy.BUFFER)
+                .subscribe(CamelReactiveStreams.get(context).getSubscriber("slowNumbers", Integer.class));
+
+        MockEndpoint endpoint = getMockEndpoint("mock:endpoint");
+        endpoint.expectedMessageCount(10);
+        endpoint.assertIsSatisfied();
+        long end = System.currentTimeMillis();
+
+        // Maximum one inflight exchange, even if multiple consumer threads are present
+        // Must take at least 50 * 10 = 500ms
+        assertTrue("Exchange completed too early", end - start >= 500);
+    }
+
+    @Test
+    public void testSlowerBackpressure() throws Exception {
+
+        long start = System.currentTimeMillis();
+        Observable.range(0, 2)
+                .toFlowable(BackpressureStrategy.BUFFER)
+                .subscribe(CamelReactiveStreams.get(context).getSubscriber("slowerNumbers", Integer.class));
+
+        MockEndpoint endpoint = getMockEndpoint("mock:endpoint");
+        endpoint.expectedMessageCount(2);
+        endpoint.assertIsSatisfied();
+        long end = System.currentTimeMillis();
+
+        // Maximum one inflight exchange, even if multiple consumer threads are present
+        // Must take at least 300 * 2 = 600ms
+        assertTrue("Exchange completed too early", end - start >= 600);
+    }
+
+    @Test
+    public void testParallelSlowBackpressure() throws Exception {
+
+        long start = System.currentTimeMillis();
+        Flowable.range(0, 40)
+                .subscribe(CamelReactiveStreams.get(context).getSubscriber("parallelSlowNumbers", Integer.class));
+
+        MockEndpoint endpoint = getMockEndpoint("mock:endpoint");
+        endpoint.expectedMessageCount(40);
+        endpoint.assertIsSatisfied();
+        long end = System.currentTimeMillis();
+
+        // Maximum 5 inflight exchanges
+        // Must take at least 100 * (40 / 5) = 800ms
+        assertTrue("Exchange completed too early", end - start >= 800);
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("reactive-streams:slowNumbers?concurrentConsumers=10&maxInflightExchanges=1")
+                        .process(x -> Thread.sleep(50))
+                        .to("mock:endpoint");
+
+                from("reactive-streams:slowerNumbers?concurrentConsumers=10&maxInflightExchanges=1")
+                        .process(x -> Thread.sleep(300))
+                        .to("mock:endpoint");
+
+                from("reactive-streams:parallelSlowNumbers?concurrentConsumers=10&maxInflightExchanges=5")
+                        .process(x -> Thread.sleep(100))
+                        .to("mock:endpoint");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicPublisherTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicPublisherTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicPublisherTest.java
new file mode 100644
index 0000000..84b15de
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicPublisherTest.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.reactive.streams;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import io.reactivex.Observable;
+import io.reactivex.disposables.Disposable;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class BasicPublisherTest extends CamelTestSupport {
+
+    @Test
+    public void testWorking() throws Exception {
+
+        new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:tick?period=5&repeatCount=30")
+                        .setBody().header(Exchange.TIMER_COUNTER)
+                        .to("reactive-streams:pub");
+            }
+        }.addRoutesToCamelContext(context);
+
+        int num = 30;
+        CountDownLatch latch = new CountDownLatch(num);
+        List<Integer> recv = new LinkedList<>();
+
+        Observable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("pub", Integer.class))
+                .doOnNext(recv::add)
+                .doOnNext(n -> latch.countDown())
+                .subscribe();
+
+        context.start();
+        latch.await(5, TimeUnit.SECONDS);
+
+        assertEquals(num, recv.size());
+        for (int i = 1; i <= num; i++) {
+            assertEquals(i, recv.get(i - 1).intValue());
+        }
+    }
+
+    @Test
+    public void testMultipleSubscriptions() throws Exception {
+
+        new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:tick?period=50")
+                        .setBody().header(Exchange.TIMER_COUNTER)
+                        .to("reactive-streams:unbounded");
+            }
+        }.addRoutesToCamelContext(context);
+
+        CountDownLatch latch1 = new CountDownLatch(5);
+        Disposable disp1 = Observable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("unbounded", Integer.class))
+                .subscribe(n -> latch1.countDown());
+
+        context.start();
+
+        // Add another subscription
+        CountDownLatch latch2 = new CountDownLatch(5);
+        Disposable disp2 = Observable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("unbounded", Integer.class))
+                .subscribe(n -> latch2.countDown());
+
+        assertTrue(latch1.await(5, TimeUnit.SECONDS));
+        assertTrue(latch2.await(5, TimeUnit.SECONDS));
+
+        // Unsubscribe both
+        disp1.dispose();
+        disp2.dispose();
+
+        // No active subscriptions, warnings expected
+        Thread.sleep(60);
+
+        // Add another subscription
+        CountDownLatch latch3 = new CountDownLatch(5);
+        Disposable disp3 = Observable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("unbounded", Integer.class))
+                .subscribe(n -> latch3.countDown());
+
+        assertTrue(latch3.await(5, TimeUnit.SECONDS));
+        disp3.dispose();
+    }
+
+    @Override
+    public boolean isUseRouteBuilder() {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicSubscriberTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicSubscriberTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicSubscriberTest.java
new file mode 100644
index 0000000..ba7bea8
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/BasicSubscriberTest.java
@@ -0,0 +1,85 @@
+/**
+ * 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.reactive.streams;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+
+public class BasicSubscriberTest extends CamelTestSupport {
+
+    @Test
+    public void testWorking() throws Exception {
+        int count = 2;
+        MockEndpoint e1 = getMockEndpoint("mock:sub1");
+        e1.expectedMinimumMessageCount(count);
+        e1.assertIsSatisfied();
+
+        MockEndpoint e2 = getMockEndpoint("mock:sub2");
+        e2.expectedMinimumMessageCount(count);
+        e2.assertIsSatisfied();
+
+        MockEndpoint e3 = getMockEndpoint("mock:sub3");
+        e3.expectedMinimumMessageCount(count);
+        e3.assertIsSatisfied();
+
+        for (int i = 0; i < count; i++) {
+            Exchange ex1 = e1.getExchanges().get(i);
+            Exchange ex2 = e2.getExchanges().get(i);
+            Exchange ex3 = e3.getExchanges().get(i);
+            assertEquals(ex1.getIn().getBody(), ex2.getIn().getBody());
+            assertEquals(ex1.getIn().getBody(), ex3.getIn().getBody());
+        }
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    protected void doPostSetup() throws Exception {
+
+        Subscriber<Integer> sub = CamelReactiveStreams.get(context()).getSubscriber("sub", Integer.class);
+        Subscriber<Integer> sub2 = CamelReactiveStreams.get(context()).getSubscriber("sub2", Integer.class);
+        Publisher<Integer> pub = CamelReactiveStreams.get(context()).getPublisher("pub", Integer.class);
+
+        pub.subscribe(sub);
+        pub.subscribe(sub2);
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("reactive-streams:sub")
+                        .to("mock:sub1");
+
+                from("reactive-streams:sub2")
+                        .to("mock:sub2");
+
+                from("timer:tick?period=50")
+                        .setBody().simple("random(500)")
+                        .wireTap("mock:sub3")
+                        .to("reactive-streams:pub");
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherConversionVerificationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherConversionVerificationTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherConversionVerificationTest.java
new file mode 100644
index 0000000..a3b1c6b
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherConversionVerificationTest.java
@@ -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.component.reactive.streams;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.tck.PublisherVerification;
+import org.reactivestreams.tck.TestEnvironment;
+
+public class CamelPublisherConversionVerificationTest extends PublisherVerification<Long> {
+
+    public CamelPublisherConversionVerificationTest() {
+        super(new TestEnvironment(2000L));
+    }
+
+    @Override
+    public Publisher<Long> createPublisher(long l) {
+
+        CamelContext context = new DefaultCamelContext();
+        RouteBuilder builder = new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:tick?delay=500&period=50&repeatCount=" + l)
+                        .setBody().simple("random(1000)")
+                        .to("reactive-streams:prod");
+            }
+        };
+
+        Publisher<Long> pub = CamelReactiveStreams.get(context).getPublisher("prod", Long.class);
+
+        try {
+            builder.addRoutesToCamelContext(context);
+            context.start();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+
+        return pub;
+    }
+
+    @Override
+    public long maxElementsFromPublisher() {
+        // It's an active publisher
+        return publisherUnableToSignalOnComplete(); // == Long.MAX_VALUE == unbounded
+    }
+
+    @Override
+    public Publisher<Long> createFailedPublisher() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherVerificationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherVerificationTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherVerificationTest.java
new file mode 100644
index 0000000..a29e05e
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelPublisherVerificationTest.java
@@ -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.component.reactive.streams;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.tck.PublisherVerification;
+import org.reactivestreams.tck.TestEnvironment;
+
+public class CamelPublisherVerificationTest extends PublisherVerification<Exchange> {
+
+    public CamelPublisherVerificationTest() {
+        super(new TestEnvironment(2000L));
+    }
+
+    @Override
+    public Publisher<Exchange> createPublisher(long l) {
+
+        CamelContext context = new DefaultCamelContext();
+        RouteBuilder builder = new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:tick?delay=500&period=50&repeatCount=" + l)
+                        .to("reactive-streams:prod");
+            }
+        };
+
+        Publisher<Exchange> pub = CamelReactiveStreams.get(context).getPublisher("prod");
+
+        try {
+            builder.addRoutesToCamelContext(context);
+            context.start();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+
+        return pub;
+    }
+
+    @Override
+    public long maxElementsFromPublisher() {
+        // It's an active publisher
+        return publisherUnableToSignalOnComplete(); // == Long.MAX_VALUE == unbounded
+    }
+
+    @Override
+    public Publisher<Exchange> createFailedPublisher() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberConversionVerificationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberConversionVerificationTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberConversionVerificationTest.java
new file mode 100644
index 0000000..8f1d157
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberConversionVerificationTest.java
@@ -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.component.reactive.streams;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.tck.SubscriberBlackboxVerification;
+import org.reactivestreams.tck.TestEnvironment;
+
+public class CamelSubscriberConversionVerificationTest extends SubscriberBlackboxVerification<Integer> {
+
+    private CamelContext context;
+
+    public CamelSubscriberConversionVerificationTest() {
+        super(new TestEnvironment(2000L));
+    }
+
+    @Override
+    public Subscriber<Integer> createSubscriber() {
+        this.context = new DefaultCamelContext();
+        RouteBuilder builder = new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("reactive-streams:sub?maxInflightExchanges=20")
+                        .to("log:INFO");
+            }
+        };
+
+        Subscriber<Integer> sub = CamelReactiveStreams.get(context).getSubscriber("sub", Integer.class);
+
+        try {
+            builder.addRoutesToCamelContext(context);
+            context.start();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+
+        return sub;
+    }
+
+    @Override
+    public Integer createElement(int element) {
+        return element;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberVerificationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberVerificationTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberVerificationTest.java
new file mode 100644
index 0000000..921f468
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/CamelSubscriberVerificationTest.java
@@ -0,0 +1,66 @@
+/**
+ * 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.reactive.streams;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.apache.camel.impl.DefaultExchange;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.tck.SubscriberBlackboxVerification;
+import org.reactivestreams.tck.TestEnvironment;
+
+public class CamelSubscriberVerificationTest extends SubscriberBlackboxVerification<Exchange> {
+
+    private CamelContext context;
+
+    public CamelSubscriberVerificationTest() {
+        super(new TestEnvironment(2000L));
+    }
+
+    @Override
+    public Subscriber<Exchange> createSubscriber() {
+        this.context = new DefaultCamelContext();
+        RouteBuilder builder = new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("reactive-streams:sub?maxInflightExchanges=20")
+                        .to("log:INFO");
+            }
+        };
+
+        Subscriber<Exchange> sub = CamelReactiveStreams.get(context).getSubscriber("sub");
+
+        try {
+            builder.addRoutesToCamelContext(context);
+            context.start();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+
+        return sub;
+    }
+
+    @Override
+    public Exchange createElement(int element) {
+        Exchange exchange = new DefaultExchange(context);
+        exchange.getIn().setBody(element);
+        return exchange;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/ConcurrentConsumersTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/ConcurrentConsumersTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/ConcurrentConsumersTest.java
new file mode 100644
index 0000000..95f6474
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/ConcurrentConsumersTest.java
@@ -0,0 +1,103 @@
+/**
+ * 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.reactive.streams;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import io.reactivex.BackpressureStrategy;
+import io.reactivex.Observable;
+
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+/**
+ * Test the behaviour of the consumer side when using a different number of consumer threads.
+ */
+public class ConcurrentConsumersTest extends CamelTestSupport {
+
+    @Test
+    public void testSingleConsumer() throws Exception {
+
+        Observable.intervalRange(0, 1000, 0, 300, TimeUnit.MICROSECONDS)
+                .toFlowable(BackpressureStrategy.BUFFER)
+                .subscribe(CamelReactiveStreams.get(context()).getSubscriber("singleConsumer", Long.class));
+
+        MockEndpoint endpoint = getMockEndpoint("mock:singleBucket");
+        endpoint.expectedMessageCount(1000);
+        endpoint.assertIsSatisfied();
+
+        Set<String> threads = endpoint.getExchanges().stream()
+                .map(x -> x.getIn().getHeader("thread", String.class))
+                .collect(Collectors.toSet());
+        assertEquals(1, threads.size());
+
+        // Ensure order is preserved when using a single consumer
+        List<Long> nums = endpoint.getExchanges().stream()
+                .map(x -> x.getIn().getBody(Long.class))
+                .collect(Collectors.toList());
+
+        long prev = -1;
+        for (long n : nums) {
+            assertEquals(prev + 1, n);
+            prev = n;
+        }
+    }
+
+    @Test
+    public void testMultipleConsumers() throws Exception {
+
+        Observable.intervalRange(0, 1000, 0, 300, TimeUnit.MICROSECONDS)
+                .toFlowable(BackpressureStrategy.BUFFER)
+                .subscribe(CamelReactiveStreams.get(context()).getSubscriber("multipleConsumers", Long.class));
+
+        MockEndpoint endpoint = getMockEndpoint("mock:multipleBucket");
+        endpoint.expectedMessageCount(1000);
+        endpoint.assertIsSatisfied();
+
+        Set<String> threads = endpoint.getExchanges().stream()
+                .map(x -> x.getIn().getHeader("thread", String.class))
+                .collect(Collectors.toSet());
+
+        assertEquals(3, threads.size());
+        // Order cannot be preserved when using multiple consumers
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+
+                from("reactive-streams:singleConsumer")
+                        .process(x -> x.getIn().setHeader("thread", Thread.currentThread().getId()))
+                        .to("mock:singleBucket");
+
+                from("reactive-streams:multipleConsumers?concurrentConsumers=3")
+                        .process(x -> x.getIn().setHeader("thread", Thread.currentThread().getId()))
+                        .to("mock:multipleBucket");
+
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/PublisherTypeConversionTest.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/PublisherTypeConversionTest.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/PublisherTypeConversionTest.java
new file mode 100644
index 0000000..f117878
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/PublisherTypeConversionTest.java
@@ -0,0 +1,88 @@
+/**
+ * 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.reactive.streams;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import io.reactivex.Observable;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.RoutesBuilder;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.test.junit4.CamelTestSupport;
+import org.junit.Test;
+
+public class PublisherTypeConversionTest extends CamelTestSupport {
+
+
+    @Test
+    public void testConversion() throws Exception {
+
+        CountDownLatch latch = new CountDownLatch(3);
+        List<Integer> integers = new LinkedList<>();
+
+        Observable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("pub", Exchange.class))
+                .map(x -> x.getIn().getBody(Integer.class))
+                .subscribe(n -> {
+                    integers.add(n);
+                    latch.countDown();
+                });
+
+        Observable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("pub"))
+                .map(x -> x.getIn().getBody(Integer.class))
+                .subscribe(n -> {
+                    integers.add(n);
+                    latch.countDown();
+                });
+
+        Observable.fromPublisher(CamelReactiveStreams.get(context).getPublisher("pub", Integer.class))
+                .subscribe(n -> {
+                    integers.add(n);
+                    latch.countDown();
+                });
+
+        context.start();
+        latch.await(5, TimeUnit.SECONDS);
+
+        assertEquals(3, integers.size());
+
+        for (int i : integers) {
+            assertEquals(123, i);
+        }
+    }
+
+    @Override
+    protected RoutesBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("timer:tick?period=50&repeatCount=1")
+                        .setBody().constant(123)
+                        .to("reactive-streams:pub");
+            }
+        };
+    }
+
+    @Override
+    public boolean isUseAdviceWith() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestPublisher.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestPublisher.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestPublisher.java
new file mode 100644
index 0000000..0165bed
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestPublisher.java
@@ -0,0 +1,110 @@
+/**
+ * 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.reactive.streams.support;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+/**
+ * A publisher for tests.
+ */
+public class TestPublisher<T> implements Publisher<T> {
+
+    private Iterable<T> data;
+
+    private long delay;
+
+    public TestPublisher(Iterable<T> data) {
+        this(data, 0L);
+    }
+
+    public TestPublisher(Iterable<T> data, long delay) {
+        this.data = data;
+        this.delay = delay;
+    }
+
+    @Override
+    public void subscribe(Subscriber<? super T> subscriber) {
+        subscriber.onSubscribe(new Subscription() {
+
+            private Iterator<T> it = data.iterator();
+
+            private AtomicLong requested = new AtomicLong(0);
+
+            private Object monitor = new Object();
+
+            @Override
+            public void request(long l) {
+                this.requested.addAndGet(l);
+
+                new Thread() {
+                    @Override
+                    public void run() {
+                        synchronized (monitor) {
+                            boolean wasNonEmpty = it.hasNext();
+                            while (requested.longValue() > 0 && it.hasNext()) {
+                                T d = it.next();
+                                requested.decrementAndGet();
+
+                                if (delay > 0) {
+                                    try {
+                                        Thread.sleep(delay);
+                                    } catch (InterruptedException ex) {
+                                    }
+                                }
+                                subscriber.onNext(d);
+                            }
+
+                            if (wasNonEmpty && !it.hasNext()) {
+                                // data cannot be added to this test publisher
+                                subscriber.onComplete();
+                            }
+                        }
+                    }
+                }.start();
+            }
+
+            @Override
+            public void cancel() {
+                synchronized (monitor) {
+                    this.requested.set(0);
+                    this.it = new Iterator<T>() {
+                        @Override
+                        public boolean hasNext() {
+                            return false;
+                        }
+
+                        @Override
+                        public T next() {
+                            throw new NoSuchElementException();
+                        }
+                    };
+                    new Thread() {
+                        public void run() {
+                            subscriber.onComplete();
+                        }
+                    }.start();
+                }
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestSubscriber.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestSubscriber.java b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestSubscriber.java
new file mode 100644
index 0000000..f49d5c9
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/java/org/apache/camel/component/reactive/streams/support/TestSubscriber.java
@@ -0,0 +1,73 @@
+/**
+ * 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.reactive.streams.support;
+
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+/**
+ * A subscriber for tests.
+ */
+public class TestSubscriber<T> implements Subscriber<T> {
+
+    protected Subscription subscription;
+
+    private long initiallyRequested;
+
+    public TestSubscriber() {
+    }
+
+    public long getInitiallyRequested() {
+        return initiallyRequested;
+    }
+
+    public void setInitiallyRequested(long initiallyRequested) {
+        this.initiallyRequested = initiallyRequested;
+    }
+
+    public void request(long exchanges) {
+        this.subscription.request(exchanges);
+    }
+
+    public void cancel() {
+        this.subscription.cancel();
+    }
+
+    @Override
+    public void onSubscribe(Subscription subscription) {
+        this.subscription = subscription;
+
+        if (initiallyRequested > 0) {
+            subscription.request(initiallyRequested);
+        }
+    }
+
+    @Override
+    public void onNext(T t) {
+
+    }
+
+    @Override
+    public void onError(Throwable throwable) {
+
+    }
+
+    @Override
+    public void onComplete() {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/test/resources/log4j2.properties
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/test/resources/log4j2.properties b/components/camel-reactive-streams/src/test/resources/log4j2.properties
new file mode 100644
index 0000000..fbab18c
--- /dev/null
+++ b/components/camel-reactive-streams/src/test/resources/log4j2.properties
@@ -0,0 +1,33 @@
+## ---------------------------------------------------------------------------
+## 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.
+## ---------------------------------------------------------------------------
+
+appender.file.type = File
+appender.file.name = file
+appender.file.fileName = target/camel-reactive-streams-test.log
+appender.file.layout.type = PatternLayout
+appender.file.layout.pattern = %d [%-15.15t] %-5p %-30.30c{1} - %m%n
+appender.stdout.type = Console
+appender.stdout.name = stdout
+appender.stdout.layout.type = PatternLayout
+appender.stdout.layout.pattern = %d [%-15.15t] %-5p %-30.30c{1} - %m%n
+rootLogger.level = INFO
+rootLogger.appenderRef.file.ref = file
+
+#rootLogger.appenderRef.stdout.ref = stdout
+#logger.debug.name = org.apache.camel.component.reactive.streams
+#logger.debug.level = DEBUG
+#logger.debug.additivity = true

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/pom.xml
----------------------------------------------------------------------
diff --git a/components/pom.xml b/components/pom.xml
index def06f5..4276240 100644
--- a/components/pom.xml
+++ b/components/pom.xml
@@ -216,6 +216,7 @@
     <module>camel-quartz2</module>
     <module>camel-quickfix</module>
     <module>camel-rabbitmq</module>
+    <module>camel-reactive-streams</module>
     <module>camel-restlet</module>
     <module>camel-rmi</module>
     <module>camel-routebox</module>

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/readme.adoc
----------------------------------------------------------------------
diff --git a/components/readme.adoc b/components/readme.adoc
index 9813764..b8fb246 100644
--- a/components/readme.adoc
+++ b/components/readme.adoc
@@ -438,6 +438,9 @@ Components
 | link:camel-rabbitmq/src/main/docs/rabbitmq-component.adoc[RabbitMQ] (camel-rabbitmq) +
 `rabbitmq:hostname:portNumber/exchangeName` | The rabbitmq component allows you produce and consume messages from RabbitMQ instances.
 
+| link:camel-reactive-streams/src/main/docs/reactive-streams-component.adoc[Reactive Streams] (camel-reactive-streams) +
+`reactive-streams:/stream` | The Camel reactive-streams endpoint.
+
 | link:camel-restlet/src/main/docs/restlet-component.adoc[Restlet] (camel-restlet) +
 `restlet:protocol:host:port/uriPattern` | Component for consuming and producing Restful resources using Restlet.
 

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/docs/user-manual/en/SUMMARY.md
----------------------------------------------------------------------
diff --git a/docs/user-manual/en/SUMMARY.md b/docs/user-manual/en/SUMMARY.md
index 54c6702..b7dd18c 100644
--- a/docs/user-manual/en/SUMMARY.md
+++ b/docs/user-manual/en/SUMMARY.md
@@ -268,6 +268,7 @@
 	* [Quartz2](quartz2-component.adoc)
 	* [QuickFix](quickfix-component.adoc)
 	* [RabbitMQ](rabbitmq-component.adoc)
+	* [Reactive Streams](reactive-streams-component.adoc)
 	* [Restlet](restlet-component.adoc)
 	* [RMI](rmi-component.adoc)
 	* [RouteBox](routebox-component.adoc)

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index ac58a34..41edd0b 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -520,6 +520,7 @@
     <rome-bundle-version>1.0_3</rome-bundle-version>
     <rome-version>1.0</rome-version>
     <rxjava-version>1.2.5</rxjava-version>
+    <rxjava2-version>2.0.4</rxjava2-version>
     <saaj-impl-version>1.3.2_2</saaj-impl-version>
     <saxon-bundle-version>9.7.0-14_1</saxon-bundle-version>
     <saxon-version>9.7.0-14</saxon-version>

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/pom.xml
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/pom.xml b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/pom.xml
new file mode 100644
index 0000000..5325d29
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/pom.xml
@@ -0,0 +1,51 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.camel</groupId>
+    <artifactId>components-starter</artifactId>
+    <version>2.19.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>camel-reactive-streams-starter</artifactId>
+  <packaging>jar</packaging>
+  <name>Spring-Boot Starter :: Camel :: Reactive Streams</name>
+  <description>Spring-Boot Starter for Camel Reactive Streams Component</description>
+  <dependencies>
+    <dependency>
+      <groupId>org.springframework.boot</groupId>
+      <artifactId>spring-boot-starter</artifactId>
+      <version>${spring-boot-version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-reactive-streams</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <!--START OF GENERATED CODE-->
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-core-starter</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-spring-boot-starter</artifactId>
+    </dependency>
+    <!--END OF GENERATED CODE-->
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentAutoConfiguration.java
----------------------------------------------------------------------
diff --git a/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentAutoConfiguration.java b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentAutoConfiguration.java
new file mode 100644
index 0000000..10f6dad
--- /dev/null
+++ b/platforms/spring-boot/components-starter/camel-reactive-streams-starter/src/main/java/org/apache/camel/component/reactive/streams/springboot/ReactiveStreamsComponentAutoConfiguration.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.reactive.streams.springboot;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.camel.CamelContext;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsComponent;
+import org.apache.camel.util.IntrospectionSupport;
+import org.springframework.boot.autoconfigure.AutoConfigureAfter;
+import org.springframework.boot.autoconfigure.condition.ConditionMessage;
+import org.springframework.boot.autoconfigure.condition.ConditionOutcome;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnBean;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnClass;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean;
+import org.springframework.boot.autoconfigure.condition.SpringBootCondition;
+import org.springframework.boot.bind.RelaxedPropertyResolver;
+import org.springframework.boot.context.properties.EnableConfigurationProperties;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ConditionContext;
+import org.springframework.context.annotation.Conditional;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.context.annotation.Lazy;
+import org.springframework.core.type.AnnotatedTypeMetadata;
+
+/**
+ * Generated by camel-package-maven-plugin - do not edit this file!
+ */
+@Configuration
+@ConditionalOnBean(type = "org.apache.camel.spring.boot.CamelAutoConfiguration")
+@Conditional(ReactiveStreamsComponentAutoConfiguration.Condition.class)
+@AutoConfigureAfter(name = "org.apache.camel.spring.boot.CamelAutoConfiguration")
+@EnableConfigurationProperties(ReactiveStreamsComponentConfiguration.class)
+public class ReactiveStreamsComponentAutoConfiguration {
+
+    @Lazy
+    @Bean(name = "reactive-streams-component")
+    @ConditionalOnClass(CamelContext.class)
+    @ConditionalOnMissingBean(ReactiveStreamsComponent.class)
+    public ReactiveStreamsComponent configureReactiveStreamsComponent(
+            CamelContext camelContext,
+            ReactiveStreamsComponentConfiguration configuration)
+            throws Exception {
+        ReactiveStreamsComponent component = new ReactiveStreamsComponent();
+        component.setCamelContext(camelContext);
+        Map<String, Object> parameters = new HashMap<>();
+        IntrospectionSupport.getProperties(configuration, parameters, null,
+                false);
+        for (Map.Entry<String, Object> entry : parameters.entrySet()) {
+            Object value = entry.getValue();
+            Class<?> paramClass = value.getClass();
+            if (paramClass.getName().endsWith("NestedConfiguration")) {
+                Class nestedClass = null;
+                try {
+                    nestedClass = (Class) paramClass.getDeclaredField(
+                            "CAMEL_NESTED_CLASS").get(null);
+                    HashMap<String, Object> nestedParameters = new HashMap<>();
+                    IntrospectionSupport.getProperties(value, nestedParameters,
+                            null, false);
+                    Object nestedProperty = nestedClass.newInstance();
+                    IntrospectionSupport.setProperties(camelContext,
+                            camelContext.getTypeConverter(), nestedProperty,
+                            nestedParameters);
+                    entry.setValue(nestedProperty);
+                } catch (NoSuchFieldException e) {
+                }
+            }
+        }
+        IntrospectionSupport.setProperties(camelContext,
+                camelContext.getTypeConverter(), component, parameters);
+        return component;
+    }
+
+    public static class Condition extends SpringBootCondition {
+        @Override
+        public ConditionOutcome getMatchOutcome(
+                ConditionContext conditionContext,
+                AnnotatedTypeMetadata annotatedTypeMetadata) {
+            boolean groupEnabled = isEnabled(conditionContext,
+                    "camel.component.", true);
+            ConditionMessage.Builder message = ConditionMessage
+                    .forCondition("camel.component.reactive-streams");
+            if (isEnabled(conditionContext,
+                    "camel.component.reactive-streams.", groupEnabled)) {
+                return ConditionOutcome.match(message.because("enabled"));
+            }
+            return ConditionOutcome.noMatch(message.because("not enabled"));
+        }
+
+        private boolean isEnabled(
+                org.springframework.context.annotation.ConditionContext context,
+                java.lang.String prefix, boolean defaultValue) {
+            RelaxedPropertyResolver resolver = new RelaxedPropertyResolver(
+                    context.getEnvironment(), prefix);
+            return resolver.getProperty("enabled", Boolean.class, defaultValue);
+        }
+    }
+}
\ No newline at end of file


[2/6] camel git commit: CAMEL-10650: some fixes after review

Posted by nf...@apache.org.
CAMEL-10650: some fixes after review


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

Branch: refs/heads/master
Commit: 40b42e656479b568d401baebe30e1dbc71c4979b
Parents: e66207a
Author: Nicola Ferraro <ni...@gmail.com>
Authored: Mon Jan 23 10:33:04 2017 +0100
Committer: Nicola Ferraro <ni...@gmail.com>
Committed: Fri Jan 27 15:31:31 2017 +0100

----------------------------------------------------------------------
 .../reactive/streams/ReactiveStreamsConsumer.java   |  2 +-
 .../reactive/streams/ReactiveStreamsEndpoint.java   |  2 +-
 .../reactive/streams/ReactiveStreamsProducer.java   |  4 +---
 .../reactive/streams/api/CamelReactiveStreams.java  | 16 ++++++++--------
 .../reactive/streams/engine/CamelPublisher.java     | 11 +++++++----
 .../reactive/streams/engine/StreamPayload.java      |  9 ---------
 .../streams/engine/UnwrappingPublisher.java         | 12 +++++++-----
 .../reactive/streams/util/ConvertingPublisher.java  | 13 ++++++++-----
 8 files changed, 33 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
index c345a2b..ca19e0a 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
@@ -71,7 +71,7 @@ public class ReactiveStreamsConsumer extends DefaultConsumer {
 
             executorService.execute(() -> this.getAsyncProcessor().process(exchange, doneSync -> {
                 if (exchange.getException() != null) {
-                    LOG.warn("Error processing the exchange " + exchange + " from reactive-streams", exchange.getException());
+                    getExceptionHandler().handleException("Error processing exchange", exchange, exchange.getException());
                 }
 
                 callback.done(doneSync);

http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
index 7d294f1..0e72c52 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
@@ -27,7 +27,7 @@ import org.apache.camel.spi.UriPath;
 /**
  * The Camel reactive-streams endpoint.
  */
-@UriEndpoint(scheme = "reactive-streams", title = "Reactive Streams", syntax = "reactive-streams:/stream",
+@UriEndpoint(scheme = "reactive-streams", title = "Reactive Streams", syntax = "reactive-streams:stream",
         consumerClass = ReactiveStreamsConsumer.class, label = "streams")
 public class ReactiveStreamsEndpoint extends DefaultEndpoint {
 

http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
index a8ec559..d74cdb1 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
@@ -43,9 +43,7 @@ public class ReactiveStreamsProducer<T> extends DefaultAsyncProducer {
                 data.setException(error);
             }
 
-            if (callback != null) {
-                callback.done(false);
-            }
+            callback.done(false);
         });
         return false;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
index dfa8185..1bafc5c 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Exchange;
@@ -40,7 +41,7 @@ public final class CamelReactiveStreams {
 
     private static final Logger LOG = LoggerFactory.getLogger(CamelReactiveStreams.class);
 
-    private static Map<CamelContext, CamelReactiveStreams> instances = new HashMap<>();
+    private static Map<CamelContext, CamelReactiveStreams> instances = new ConcurrentHashMap<>();
 
     private CamelReactiveStreamsService service;
 
@@ -48,17 +49,16 @@ public final class CamelReactiveStreams {
         this.service = service;
     }
 
-    public static synchronized CamelReactiveStreams get(CamelContext context) {
-        if (!instances.containsKey(context)) {
+    public static CamelReactiveStreams get(CamelContext context) {
+        instances.computeIfAbsent(context, ctx -> {
             CamelReactiveStreamsService service = resolveReactiveStreamsService(context);
             try {
-                context.addService(service, true, true);
+                ctx.addService(service, true, true);
             } catch (Exception ex) {
                 throw new IllegalStateException("Cannot add the CamelReactiveStreamsService to the Camel context", ex);
             }
-
-            instances.put(context, new CamelReactiveStreams(service));
-        }
+            return new CamelReactiveStreams(service);
+        });
 
         return instances.get(context);
     }
@@ -81,7 +81,7 @@ public final class CamelReactiveStreams {
             try {
                 service = serviceClass.newInstance();
                 LOG.info("Created reactive stream service from class: " + serviceClass.getName());
-            } catch (InstantiationException | IllegalAccessException e) {
+            } catch (Exception e) {
                 LOG.debug("Unable to create a reactive stream service of class " + serviceClass.getName(), e);
             }
         }

http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java
index bf675fb..6a30625 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java
@@ -70,15 +70,18 @@ public class CamelPublisher implements Publisher<StreamPayload<Exchange>>, AutoC
     public void publish(StreamPayload<Exchange> data) {
         // freeze the subscriptions
         List<CamelSubscription> subs = new LinkedList<>(subscriptions);
+
         DispatchCallback<Exchange> originalCallback = data.getCallback();
         if (originalCallback != null && subs.size() > 0) {
-            // Notify processing once if multiple subscribers are present
-            AtomicInteger counter = new AtomicInteger(0);
+            // When multiple subscribers have an active subscription,
+            // we aknowledge the exchange once it has been delivered to every
+            // subscriber (or their subscription is cancelled)
+            AtomicInteger counter = new AtomicInteger(subs.size());
+            // Use just the first exception in the callback when multiple exceptions are thrown
             AtomicReference<Throwable> thrown = new AtomicReference<>(null);
             data = new StreamPayload<>(data.getItem(), (ex, error) -> {
-                int status = counter.incrementAndGet();
                 thrown.compareAndSet(null, error);
-                if (status == subs.size()) {
+                if (counter.decrementAndGet() == 0) {
                     originalCallback.processed(ex, thrown.get());
                 }
             });

http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java
index b592c40..5a1fcbe 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java
@@ -40,13 +40,4 @@ public class StreamPayload<D> {
         return callback;
     }
 
-
-    @Override
-    public String toString() {
-        final StringBuilder sb = new StringBuilder("StreamPayload{");
-        sb.append("item=").append(item);
-        sb.append(", callback=").append(callback);
-        sb.append('}');
-        return sb.toString();
-    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java
index 1cf73cc..85c87a2 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java
@@ -46,16 +46,18 @@ public class UnwrappingPublisher<R> implements Publisher<R> {
             private Subscription subscription;
 
             @Override
-            public void onSubscribe(Subscription subscription) {
-                if (subscription == null) {
+            public void onSubscribe(Subscription newSubscription) {
+                if (newSubscription == null) {
                     throw new NullPointerException("subscription is null");
+                } else if (newSubscription == this.subscription) {
+                    throw new IllegalArgumentException("already subscribed to the subscription: " + newSubscription);
                 }
 
                 if (this.subscription != null) {
-                    subscription.cancel();
+                    newSubscription.cancel();
                 } else {
-                    this.subscription = subscription;
-                    subscriber.onSubscribe(subscription);
+                    this.subscription = newSubscription;
+                    subscriber.onSubscribe(newSubscription);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/40b42e65/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java
index e8c10dd..12ed7df 100644
--- a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java
@@ -53,15 +53,18 @@ public class ConvertingPublisher<R> implements Publisher<R> {
             private Subscription subscription;
 
             @Override
-            public void onSubscribe(Subscription subscription) {
-                if (subscription == null) {
+            public void onSubscribe(Subscription newSubscription) {
+                if (newSubscription == null) {
                     throw new NullPointerException("subscription is null");
+                } else if (newSubscription == this.subscription) {
+                    throw new IllegalArgumentException("already subscribed to the subscription: " + newSubscription);
                 }
+
                 if (this.subscription != null) {
-                    subscription.cancel();
+                    newSubscription.cancel();
                 } else {
-                    this.subscription = subscription;
-                    subscriber.onSubscribe(subscription);
+                    this.subscription = newSubscription;
+                    subscriber.onSubscribe(newSubscription);
                 }
             }
 


[6/6] camel git commit: CAMEL-10612: Reactive streams

Posted by nf...@apache.org.
CAMEL-10612: Reactive streams

CAMEL-10612: First complete implementation

CAMEL-10612: Implemented subscriber verification and rx implementation

CAMEL-10612: Support switching to a different implementation

CAMEL-10612: Added internal engine configuration

CAMEL-10612: Make sure backpressure can be combined with throttling route policy

CAMEL-10612: more consistent structure

CAMEL-10612: fixes and test updates

CAMEL-10612: enhanced publisher backpressure with mixed slow and fast subscribers

CAMEL-10612: fixed issue with route restart

CAMEL-10612: fixed stream completion

CAMEL-10650: added backpressure strategy

CAMEL-10650: added javadoc

CAMEL-10650: rebase with master

CAMEL-10650: source check


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

Branch: refs/heads/master
Commit: e66207a9185df54ac7e67de027dc68f85f1ffdce
Parents: 353eeef
Author: Nicola Ferraro <ni...@gmail.com>
Authored: Wed Dec 21 18:59:01 2016 +0100
Committer: Nicola Ferraro <ni...@gmail.com>
Committed: Fri Jan 27 15:31:31 2017 +0100

----------------------------------------------------------------------
 components/camel-reactive-streams/pom.xml       |  96 +++++++
 .../ReactiveStreamsBackpressureStrategy.java    |  40 +++
 .../streams/ReactiveStreamsComponent.java       |  69 +++++
 .../streams/ReactiveStreamsConsumer.java        |  92 +++++++
 .../streams/ReactiveStreamsEndpoint.java        |  97 +++++++
 .../streams/ReactiveStreamsProducer.java        |  59 +++++
 .../streams/api/CamelReactiveStreams.java       | 166 ++++++++++++
 .../api/CamelReactiveStreamsService.java        | 114 +++++++++
 .../reactive/streams/api/DispatchCallback.java  |  34 +++
 .../reactive/streams/engine/CamelPublisher.java | 105 ++++++++
 .../engine/CamelReactiveStreamsServiceImpl.java | 133 ++++++++++
 .../streams/engine/CamelSubscriber.java         | 177 +++++++++++++
 .../streams/engine/CamelSubscription.java       | 252 +++++++++++++++++++
 .../ReactiveStreamsEngineConfiguration.java     |  65 +++++
 .../reactive/streams/engine/StreamPayload.java  |  52 ++++
 .../streams/engine/UnwrappingPublisher.java     |  87 +++++++
 .../streams/util/ConvertingPublisher.java       | 113 +++++++++
 .../streams/util/ConvertingSubscriber.java      |  75 ++++++
 .../src/main/resources/META-INF/LICENSE.txt     | 203 +++++++++++++++
 .../src/main/resources/META-INF/NOTICE.txt      |  11 +
 .../org/apache/camel/component/reactive-streams |  18 ++
 .../BackpressurePublisherRoutePolicyTest.java   | 165 ++++++++++++
 .../streams/BackpressureStrategyTest.java       | 164 ++++++++++++
 .../streams/BackpressureSubscriberTest.java     | 107 ++++++++
 .../reactive/streams/BasicPublisherTest.java    | 112 +++++++++
 .../reactive/streams/BasicSubscriberTest.java   |  85 +++++++
 ...amelPublisherConversionVerificationTest.java |  68 +++++
 .../streams/CamelPublisherVerificationTest.java |  68 +++++
 ...melSubscriberConversionVerificationTest.java |  62 +++++
 .../CamelSubscriberVerificationTest.java        |  66 +++++
 .../streams/ConcurrentConsumersTest.java        | 103 ++++++++
 .../streams/PublisherTypeConversionTest.java    |  88 +++++++
 .../reactive/streams/support/TestPublisher.java | 110 ++++++++
 .../streams/support/TestSubscriber.java         |  73 ++++++
 .../src/test/resources/log4j2.properties        |  33 +++
 components/pom.xml                              |   1 +
 components/readme.adoc                          |   3 +
 docs/user-manual/en/SUMMARY.md                  |   1 +
 parent/pom.xml                                  |   1 +
 .../camel-reactive-streams-starter/pom.xml      |  51 ++++
 ...activeStreamsComponentAutoConfiguration.java | 112 +++++++++
 .../ReactiveStreamsComponentConfiguration.java  | 100 ++++++++
 .../src/main/resources/META-INF/LICENSE.txt     | 203 +++++++++++++++
 .../src/main/resources/META-INF/NOTICE.txt      |  11 +
 ...dditional-spring-configuration-metadata.json |  10 +
 .../main/resources/META-INF/spring.factories    |  19 ++
 .../src/main/resources/META-INF/spring.provides |  18 ++
 .../spring-boot/components-starter/pom.xml      |   1 +
 48 files changed, 3893 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/pom.xml
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/pom.xml b/components/camel-reactive-streams/pom.xml
new file mode 100644
index 0000000..2357e70
--- /dev/null
+++ b/components/camel-reactive-streams/pom.xml
@@ -0,0 +1,96 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.camel</groupId>
+        <artifactId>components</artifactId>
+        <version>2.19.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>camel-reactive-streams</artifactId>
+    <packaging>jar</packaging>
+    <name>Camel :: Reactive Streams</name>
+    <description>Camel Reactive Streams Component</description>
+
+    <properties>
+        <camel.osgi.export.pkg>org.apache.camel.component.reactive.streams.*</camel.osgi.export.pkg>
+    </properties>
+
+    <dependencies>
+
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.reactivestreams</groupId>
+            <artifactId>reactive-streams</artifactId>
+            <version>1.0.0</version>
+        </dependency>
+
+        <!-- test dependencies -->
+        <dependency>
+            <groupId>org.reactivestreams</groupId>
+            <artifactId>reactive-streams-tck</artifactId>
+            <version>1.0.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>io.reactivex.rxjava2</groupId>
+            <artifactId>rxjava</artifactId>
+            <version>${rxjava2-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-test</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-slf4j-impl</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <version>${mockito-version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsBackpressureStrategy.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsBackpressureStrategy.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsBackpressureStrategy.java
new file mode 100644
index 0000000..823a7b8
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsBackpressureStrategy.java
@@ -0,0 +1,40 @@
+/**
+ * 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.reactive.streams;
+
+/**
+ * A list of possible backpressure strategy to use when the emission of upstream items cannot respect backpressure.
+ */
+public enum ReactiveStreamsBackpressureStrategy {
+
+    /**
+     * Buffers <em>all</em> onNext values until the downstream consumes it.
+     */
+    BUFFER,
+
+    /**
+     * Drops the most recent onNext value if the downstream can't keep up.
+     */
+    DROP,
+
+    /**
+     * Keeps only the latest onNext value, overwriting any previous value if the
+     * downstream can't keep up.
+     */
+    LATEST
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsComponent.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsComponent.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsComponent.java
new file mode 100644
index 0000000..5801877
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsComponent.java
@@ -0,0 +1,69 @@
+/**
+ * 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.reactive.streams;
+
+import java.util.Map;
+
+import org.apache.camel.Endpoint;
+import org.apache.camel.component.reactive.streams.engine.ReactiveStreamsEngineConfiguration;
+import org.apache.camel.impl.UriEndpointComponent;
+
+/**
+ * The Camel reactive-streams component.
+ */
+public class ReactiveStreamsComponent extends UriEndpointComponent {
+
+    private ReactiveStreamsEngineConfiguration internalEngineConfiguration = new ReactiveStreamsEngineConfiguration();
+
+    private ReactiveStreamsBackpressureStrategy backpressureStrategy = ReactiveStreamsBackpressureStrategy.BUFFER;
+
+    public ReactiveStreamsComponent() {
+        super(ReactiveStreamsEndpoint.class);
+    }
+
+    @Override
+    protected Endpoint createEndpoint(String uri, String remaining, Map<String, Object> parameters) throws Exception {
+        ReactiveStreamsEndpoint endpoint = new ReactiveStreamsEndpoint(uri, this);
+        endpoint.setStream(remaining);
+        setProperties(endpoint, parameters);
+
+        return endpoint;
+    }
+
+    public ReactiveStreamsEngineConfiguration getInternalEngineConfiguration() {
+        return internalEngineConfiguration;
+    }
+
+    /**
+     * Configures the internal engine for Reactive Streams.
+     */
+    public void setInternalEngineConfiguration(ReactiveStreamsEngineConfiguration internalEngineConfiguration) {
+        this.internalEngineConfiguration = internalEngineConfiguration;
+    }
+
+    public ReactiveStreamsBackpressureStrategy getBackpressureStrategy() {
+        return backpressureStrategy;
+    }
+
+    /**
+     * The backpressure strategy to use when pushing events to a slow subscriber.
+     */
+    public void setBackpressureStrategy(ReactiveStreamsBackpressureStrategy backpressureStrategy) {
+        this.backpressureStrategy = backpressureStrategy;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
new file mode 100644
index 0000000..c345a2b
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsConsumer.java
@@ -0,0 +1,92 @@
+/**
+ * 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.reactive.streams;
+
+import java.util.concurrent.ExecutorService;
+
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.impl.DefaultConsumer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Camel reactive-streams consumer.
+ */
+public class ReactiveStreamsConsumer extends DefaultConsumer {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ReactiveStreamsConsumer.class);
+
+    private ReactiveStreamsEndpoint endpoint;
+
+    private ExecutorService executor;
+
+    public ReactiveStreamsConsumer(ReactiveStreamsEndpoint endpoint, Processor processor) {
+        super(endpoint, processor);
+        this.endpoint = endpoint;
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        super.doStart();
+
+        int poolSize = endpoint.getConcurrentConsumers();
+        if (executor == null) {
+            executor = getEndpoint().getCamelContext().getExecutorServiceManager().newFixedThreadPool(this, getEndpoint().getEndpointUri(), poolSize);
+        }
+
+        CamelReactiveStreams.get(endpoint.getCamelContext()).getService().attachConsumer(endpoint.getStream(), this);
+    }
+
+    @Override
+    protected void doStop() throws Exception {
+        super.doStop();
+        CamelReactiveStreams.get(endpoint.getCamelContext()).getService().detachConsumer(endpoint.getStream());
+
+        if (executor != null) {
+            endpoint.getCamelContext().getExecutorServiceManager().shutdownNow(executor);
+            executor = null;
+        }
+    }
+
+    public boolean process(Exchange exchange, AsyncCallback callback) {
+        ExecutorService executorService = this.executor;
+        if (executorService != null && this.isRunAllowed()) {
+
+            executorService.execute(() -> this.getAsyncProcessor().process(exchange, doneSync -> {
+                if (exchange.getException() != null) {
+                    LOG.warn("Error processing the exchange " + exchange + " from reactive-streams", exchange.getException());
+                }
+
+                callback.done(doneSync);
+            }));
+            return false;
+
+        } else {
+            LOG.warn("Consumer not ready to process exchanges. The exchange {} will be discarded", exchange);
+            callback.done(true);
+            return true;
+        }
+    }
+
+    @Override
+    public ReactiveStreamsEndpoint getEndpoint() {
+        return endpoint;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
new file mode 100644
index 0000000..7d294f1
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsEndpoint.java
@@ -0,0 +1,97 @@
+/**
+ * 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.reactive.streams;
+
+import org.apache.camel.Consumer;
+import org.apache.camel.Processor;
+import org.apache.camel.Producer;
+import org.apache.camel.impl.DefaultEndpoint;
+import org.apache.camel.spi.UriEndpoint;
+import org.apache.camel.spi.UriParam;
+import org.apache.camel.spi.UriPath;
+
+/**
+ * The Camel reactive-streams endpoint.
+ */
+@UriEndpoint(scheme = "reactive-streams", title = "Reactive Streams", syntax = "reactive-streams:/stream",
+        consumerClass = ReactiveStreamsConsumer.class, label = "streams")
+public class ReactiveStreamsEndpoint extends DefaultEndpoint {
+
+    @UriPath
+    private String stream;
+
+    @UriParam(label = "consumer", defaultValue = "128")
+    private Integer maxInflightExchanges = 128;
+
+    @UriParam(label = "consumer", defaultValue = "1")
+    private int concurrentConsumers = 1;
+
+    public ReactiveStreamsEndpoint(String endpointUri, ReactiveStreamsComponent component) {
+        super(endpointUri, component);
+    }
+
+    @Override
+    public boolean isSingleton() {
+        return true;
+    }
+
+    @Override
+    public Producer createProducer() throws Exception {
+        return new ReactiveStreamsProducer(this, stream);
+    }
+
+    @Override
+    public Consumer createConsumer(Processor processor) throws Exception {
+        return new ReactiveStreamsConsumer(this, processor);
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    /**
+     * Name of the stream channel used by the endpoint to exchange messages.
+     */
+    public void setStream(String stream) {
+        this.stream = stream;
+    }
+
+    public Integer getMaxInflightExchanges() {
+        return maxInflightExchanges;
+    }
+
+    /**
+     * Maximum number of exchanges concurrently being processed by Camel.
+     * This parameter controls backpressure on the stream.
+     * Setting a non-positive value will disable backpressure.
+     */
+    public void setMaxInflightExchanges(Integer maxInflightExchanges) {
+        this.maxInflightExchanges = maxInflightExchanges;
+    }
+
+    public int getConcurrentConsumers() {
+        return concurrentConsumers;
+    }
+
+    /**
+     * Number of threads used to process exchanges in the Camel route.
+     */
+    public void setConcurrentConsumers(int concurrentConsumers) {
+        this.concurrentConsumers = concurrentConsumers;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
new file mode 100644
index 0000000..a8ec559
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/ReactiveStreamsProducer.java
@@ -0,0 +1,59 @@
+/**
+ * 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.reactive.streams;
+
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.Exchange;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreams;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreamsService;
+import org.apache.camel.impl.DefaultAsyncProducer;
+
+/**
+ * The Camel reactive-streams producer.
+ */
+public class ReactiveStreamsProducer<T> extends DefaultAsyncProducer {
+
+    private String name;
+
+    private CamelReactiveStreamsService service;
+
+    public ReactiveStreamsProducer(ReactiveStreamsEndpoint endpoint, String name) {
+        super(endpoint);
+        this.name = name;
+    }
+
+    @Override
+    public boolean process(Exchange exchange, AsyncCallback callback) {
+        service.process(name, exchange, (data, error) -> {
+            if (error != null) {
+                data.setException(error);
+            }
+
+            if (callback != null) {
+                callback.done(false);
+            }
+        });
+        return false;
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        super.doStart();
+        this.service = CamelReactiveStreams.get(getEndpoint().getCamelContext()).getService();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
new file mode 100644
index 0000000..dfa8185
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreams.java
@@ -0,0 +1,166 @@
+/**
+ * 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.reactive.streams.api;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.component.reactive.streams.engine.CamelReactiveStreamsServiceImpl;
+import org.apache.camel.spi.FactoryFinder;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the main entry-point for getting Camel streams associate to reactive-streams endpoints.
+ *
+ * It delegates main methods to an instance of {@link CamelReactiveStreamsService}. This component provides
+ * a default implementation that can be overridden in a 'META-INF/services/reactive-streams/reactiveStreamsService' file.
+ */
+public final class CamelReactiveStreams {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CamelReactiveStreams.class);
+
+    private static Map<CamelContext, CamelReactiveStreams> instances = new HashMap<>();
+
+    private CamelReactiveStreamsService service;
+
+    private CamelReactiveStreams(CamelReactiveStreamsService service) {
+        this.service = service;
+    }
+
+    public static synchronized CamelReactiveStreams get(CamelContext context) {
+        if (!instances.containsKey(context)) {
+            CamelReactiveStreamsService service = resolveReactiveStreamsService(context);
+            try {
+                context.addService(service, true, true);
+            } catch (Exception ex) {
+                throw new IllegalStateException("Cannot add the CamelReactiveStreamsService to the Camel context", ex);
+            }
+
+            instances.put(context, new CamelReactiveStreams(service));
+        }
+
+        return instances.get(context);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static CamelReactiveStreamsService resolveReactiveStreamsService(CamelContext context) {
+        Class<? extends CamelReactiveStreamsService> serviceClass = null;
+        try {
+            FactoryFinder finder = context.getFactoryFinder("META-INF/services/reactive-streams/");
+            LOG.trace("Using FactoryFinder: {}", finder);
+            serviceClass = (Class<? extends CamelReactiveStreamsService>) finder.findClass("reactiveStreamsService");
+        } catch (ClassNotFoundException e) {
+            LOG.trace("'reactive.streams.service.class' not found", e);
+        } catch (IOException e) {
+            LOG.trace("No reactive stream service defined in 'META-INF/services/org/apache/camel/component/'", e);
+        }
+
+        CamelReactiveStreamsService service = null;
+        if (serviceClass != null) {
+            try {
+                service = serviceClass.newInstance();
+                LOG.info("Created reactive stream service from class: " + serviceClass.getName());
+            } catch (InstantiationException | IllegalAccessException e) {
+                LOG.debug("Unable to create a reactive stream service of class " + serviceClass.getName(), e);
+            }
+        }
+
+        if (service == null) {
+            LOG.info("Using default reactive stream service");
+            service = new CamelReactiveStreamsServiceImpl();
+        }
+
+        return service;
+    }
+
+    /**
+     * Allows retrieving the service responsible for binding camel routes to streams.
+     *
+     * @return the stream service
+     */
+    public CamelReactiveStreamsService getService() {
+        return service;
+    }
+
+    /**
+     * Returns the publisher associated to the given stream name.
+     * A publisher can be used to push Camel exchanges to reactive-streams subscribers.
+     *
+     * @param name the stream name
+     * @return the stream publisher
+     */
+    public Publisher<Exchange> getPublisher(String name) {
+        Objects.requireNonNull(name, "name cannot be null");
+
+        return service.getPublisher(name);
+    }
+
+    /**
+     * Returns the publisher associated to the given stream name.
+     * A publisher can be used to push Camel exchange to external reactive-streams subscribers.
+     *
+     * The publisher converts automatically exchanges to the given type.
+     *
+     * @param name the stream name
+     * @param type the type of the emitted items
+     * @param <T> the type of items emitted by the publisher
+     * @return the publisher associated to the stream
+     */
+    public <T> Publisher<T> getPublisher(String name, Class<T> type) {
+        Objects.requireNonNull(name, "name cannot be null");
+
+        return service.getPublisher(name, type);
+    }
+
+    /**
+     * Returns the subscriber associated to the given stream name.
+     * A subscriber can be used to push items coming from external reactive-streams publishers to Camel routes.
+     *
+     * @param name the stream name
+     * @return the subscriber associated with the stream
+     */
+    public Subscriber<Exchange> getSubscriber(String name) {
+        Objects.requireNonNull(name, "name cannot be null");
+
+        return service.getSubscriber(name);
+    }
+
+    /**
+     * Returns the subscriber associated to the given stream name.
+     * A subscriber can be used to push items coming from external reactive-streams publishers to Camel routes.
+     *
+     * The subscriber converts automatically items of the given type to exchanges before pushing them.
+     *
+     * @param name the stream name
+     * @param type the publisher converts automatically exchanges to the given type.
+     * @param <T> the type of items accepted by the subscriber
+     * @return the subscriber associated with the stream
+     */
+    public <T> Subscriber<T> getSubscriber(String name, Class<T> type) {
+        Objects.requireNonNull(name, "name cannot be null");
+
+        return service.getSubscriber(name, type);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
new file mode 100644
index 0000000..147ece7
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/CamelReactiveStreamsService.java
@@ -0,0 +1,114 @@
+/**
+ * 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.reactive.streams.api;
+
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Exchange;
+import org.apache.camel.Service;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsConsumer;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+
+/**
+ * The interface to which any implementation of the reactive-streams engine should comply.
+ */
+public interface CamelReactiveStreamsService extends CamelContextAware, Service {
+
+    /*
+     * Main API methods.
+     */
+
+    /**
+     * Returns the publisher associated to the given stream name.
+     * A publisher can be used to push Camel exchanges to reactive-streams subscribers.
+     *
+     * @param name the stream name
+     * @return the stream publisher
+     */
+    Publisher<Exchange> getPublisher(String name);
+
+    /**
+     * Returns the publisher associated to the given stream name.
+     * A publisher can be used to push Camel exchange to external reactive-streams subscribers.
+     *
+     * The publisher converts automatically exchanges to the given type.
+     *
+     * @param name the stream name
+     * @param type the type of the emitted items
+     * @param <T> the type of items emitted by the publisher
+     * @return the publisher associated to the stream
+     */
+    <T> Publisher<T> getPublisher(String name, Class<T> type);
+
+    /**
+     * Returns the subscriber associated to the given stream name.
+     * A subscriber can be used to push items coming from external reactive-streams publishers to Camel routes.
+     *
+     * @param name the stream name
+     * @return the subscriber associated with the stream
+     */
+    Subscriber<Exchange> getSubscriber(String name);
+
+    /**
+     * Returns the subscriber associated to the given stream name.
+     * A subscriber can be used to push items coming from external reactive-streams publishers to Camel routes.
+     *
+     * The subscriber converts automatically items of the given type to exchanges before pushing them.
+     *
+     * @param name the stream name
+     * @param type the publisher converts automatically exchanges to the given type.
+     * @param <T> the type of items accepted by the subscriber
+     * @return the subscriber associated with the stream
+     */
+    <T> Subscriber<T> getSubscriber(String name, Class<T> type);
+
+    /*
+     * Methods for producers.
+     */
+
+    /**
+     * Sends the exchange to all active subscriptions on the given stream.
+     * The callback is used to signal that the exchange has been delivered to the subscribers.
+     *
+     * @param name the stream name
+     * @param exchange the exchange to be forwarded to the external subscribers
+     * @param callback the callback that signals the delivery of the exchange
+     */
+    void process(String name, Exchange exchange, DispatchCallback<Exchange> callback);
+
+    /*
+     * Methods for consumers.
+     */
+
+    /**
+     * Associate the subscriber of the stream with the given name to a specific Camel consumer.
+     * This method is used to bind a Camel route to a reactive stream.
+     *
+     * @param name the stream name
+     * @param consumer the consumer of the route
+     * @throws IllegalStateException if another consumer is already associated with the given stream name
+     */
+    void attachConsumer(String name, ReactiveStreamsConsumer consumer);
+
+    /**
+     * Deassociate the existing consumer from the given stream.
+     *
+     * @param name the stream name
+     */
+    void detachConsumer(String name);
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java
new file mode 100644
index 0000000..c07f3d3
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/api/DispatchCallback.java
@@ -0,0 +1,34 @@
+/**
+ * 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.reactive.streams.api;
+
+/**
+ * A callback used to signal when a item coming from a Camel route has been delivered to the external stream processor.
+ */
+public interface DispatchCallback<T> {
+
+    /**
+     * Signals the delivery of the item.
+     * If the item cannot be delivered (no subscribers registered, conversion error)
+     * the related {@link Throwable} is specified as parameter.
+     *
+     * @param data the item
+     * @param error the error occurred, if any
+     */
+    void processed(T data, Throwable error);
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java
new file mode 100644
index 0000000..bf675fb
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelPublisher.java
@@ -0,0 +1,105 @@
+/**
+ * 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.reactive.streams.engine;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsBackpressureStrategy;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsComponent;
+import org.apache.camel.component.reactive.streams.api.DispatchCallback;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Camel publisher. It forwards Camel exchanges to external reactive-streams subscribers.
+ */
+public class CamelPublisher implements Publisher<StreamPayload<Exchange>>, AutoCloseable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CamelPublisher.class);
+
+    private ExecutorService workerPool;
+
+    private String name;
+
+    private ReactiveStreamsBackpressureStrategy backpressureStrategy;
+
+    private List<CamelSubscription> subscriptions = new CopyOnWriteArrayList<>();
+
+    public CamelPublisher(ExecutorService workerPool, CamelContext context, String name) {
+        this.workerPool = workerPool;
+        this.backpressureStrategy = ((ReactiveStreamsComponent) context.getComponent("reactive-streams")).getBackpressureStrategy();
+        this.name = name;
+    }
+
+    @Override
+    public void subscribe(Subscriber<? super StreamPayload<Exchange>> subscriber) {
+        Objects.requireNonNull(subscriber, "subscriber must not be null");
+        CamelSubscription sub = new CamelSubscription(workerPool, this, this.backpressureStrategy, subscriber);
+        this.subscriptions.add(sub);
+        subscriber.onSubscribe(sub);
+    }
+
+    public void unsubscribe(CamelSubscription subscription) {
+        subscriptions.remove(subscription);
+    }
+
+    public void publish(StreamPayload<Exchange> data) {
+        // freeze the subscriptions
+        List<CamelSubscription> subs = new LinkedList<>(subscriptions);
+        DispatchCallback<Exchange> originalCallback = data.getCallback();
+        if (originalCallback != null && subs.size() > 0) {
+            // Notify processing once if multiple subscribers are present
+            AtomicInteger counter = new AtomicInteger(0);
+            AtomicReference<Throwable> thrown = new AtomicReference<>(null);
+            data = new StreamPayload<>(data.getItem(), (ex, error) -> {
+                int status = counter.incrementAndGet();
+                thrown.compareAndSet(null, error);
+                if (status == subs.size()) {
+                    originalCallback.processed(ex, thrown.get());
+                }
+            });
+        }
+
+        if (subs.size() > 0) {
+            LOG.debug("Exchange published to {} subscriptions for the stream {}: {}", subs.size(), name, data.getItem());
+            // at least one subscriber
+            for (CamelSubscription sub : subs) {
+                sub.publish(data);
+            }
+        } else {
+            data.getCallback().processed(data.getItem(), new IllegalStateException("The stream has no active subscriptions"));
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        for (CamelSubscription sub : subscriptions) {
+            sub.signalCompletion();
+        }
+        subscriptions.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
new file mode 100644
index 0000000..fb6b693
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelReactiveStreamsServiceImpl.java
@@ -0,0 +1,133 @@
+/**
+ * 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.reactive.streams.engine;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsComponent;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsConsumer;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreamsService;
+import org.apache.camel.component.reactive.streams.api.DispatchCallback;
+import org.apache.camel.component.reactive.streams.util.ConvertingPublisher;
+import org.apache.camel.component.reactive.streams.util.ConvertingSubscriber;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+
+
+public class CamelReactiveStreamsServiceImpl implements CamelReactiveStreamsService {
+
+    private CamelContext context;
+
+    private ExecutorService workerPool;
+
+    private Map<String, CamelPublisher> publishers = new HashMap<>();
+
+    private final Map<String, CamelSubscriber> subscribers = new HashMap<>();
+
+    public CamelReactiveStreamsServiceImpl() {
+    }
+
+    @Override
+    public void start() throws Exception {
+        ReactiveStreamsComponent component = context.getComponent("reactive-streams", ReactiveStreamsComponent.class);
+        ReactiveStreamsEngineConfiguration config = component.getInternalEngineConfiguration();
+        this.workerPool = context.getExecutorServiceManager().newThreadPool(this, config.getThreadPoolName(), config.getThreadPoolMinSize(), config.getThreadPoolMaxSize());
+    }
+
+    @Override
+    public void stop() throws Exception {
+        if (this.workerPool != null) {
+            context.getExecutorServiceManager().shutdownNow(this.workerPool);
+        }
+    }
+
+    @Override
+    public Publisher<Exchange> getPublisher(String name) {
+        return new UnwrappingPublisher<>(getPayloadPublisher(name));
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T> Publisher<T> getPublisher(String name, Class<T> cls) {
+        if (Exchange.class.equals(cls)) {
+            return (Publisher<T>) getPublisher(name);
+        }
+
+        return new ConvertingPublisher<T>(getPublisher(name), cls);
+    }
+
+    @Override
+    public CamelSubscriber getSubscriber(String name) {
+        synchronized (this) {
+            if (!subscribers.containsKey(name)) {
+                CamelSubscriber sub = new CamelSubscriber(name);
+                subscribers.put(name, sub);
+            }
+            return subscribers.get(name);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T> Subscriber<T> getSubscriber(String name, Class<T> type) {
+        if (Exchange.class.equals(type)) {
+            return (Subscriber<T>) getSubscriber(name);
+        }
+
+        return new ConvertingSubscriber<T>(getSubscriber(name), getCamelContext());
+    }
+
+    @Override
+    public void process(String name, Exchange exchange, DispatchCallback<Exchange> callback) {
+        StreamPayload<Exchange> payload = new StreamPayload<>(exchange, callback);
+        getPayloadPublisher(name).publish(payload);
+    }
+
+
+    private CamelPublisher getPayloadPublisher(String name) {
+        synchronized (this) {
+            if (!publishers.containsKey(name)) {
+                CamelPublisher publisher = new CamelPublisher(this.workerPool, this.context, name);
+                publishers.put(name, publisher);
+            }
+
+            return publishers.get(name);
+        }
+    }
+
+    @Override
+    public void attachConsumer(String name, ReactiveStreamsConsumer consumer) {
+        getSubscriber(name).attachConsumer(consumer);
+    }
+
+    @Override
+    public void detachConsumer(String name) {
+        getSubscriber(name).detachConsumer();
+    }
+
+    @Override
+    public void setCamelContext(CamelContext camelContext) {
+        this.context = camelContext;
+    }
+
+    @Override
+    public CamelContext getCamelContext() {
+        return this.context;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscriber.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscriber.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscriber.java
new file mode 100644
index 0000000..3b7e5d5
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscriber.java
@@ -0,0 +1,177 @@
+/**
+ * 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.reactive.streams.engine;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsConsumer;
+import org.apache.camel.component.reactive.streams.api.CamelReactiveStreamsService;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Camel subscriber. It bridges messages from reactive streams to Camel routes.
+ */
+public class CamelSubscriber implements Subscriber<Exchange>, Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CamelSubscriber.class);
+
+    /**
+     * Enough to be considered unbounded. Requests are refilled once completed.
+     */
+    private static final long MAX_INFLIGHT_UNBOUNDED = Long.MAX_VALUE / 2;
+
+    private ReactiveStreamsConsumer consumer;
+
+    private Subscription subscription;
+
+    private String name;
+
+    private long requested;
+
+    private long inflightCount;
+
+    public CamelSubscriber(String name) {
+        this.name = name;
+    }
+
+    public void attachConsumer(ReactiveStreamsConsumer consumer) {
+        synchronized (this) {
+            if (this.consumer != null) {
+                throw new IllegalStateException("A consumer is already attached on stream '" + name + "'");
+            }
+            this.consumer = consumer;
+        }
+        refill();
+    }
+
+    public void detachConsumer() {
+        synchronized (this) {
+            this.consumer = null;
+        }
+    }
+
+    @Override
+    public void onSubscribe(Subscription subscription) {
+        if (subscription == null) {
+            throw new NullPointerException("subscription is null for stream '" + name + "'");
+        }
+
+        boolean allowed = true;
+        synchronized (this) {
+            if (this.subscription != null) {
+                allowed = false;
+            } else {
+                this.subscription = subscription;
+            }
+        }
+
+        if (!allowed) {
+            subscription.cancel();
+        } else {
+            refill();
+        }
+    }
+
+    @Override
+    public void onNext(Exchange exchange) {
+        if (exchange == null) {
+            throw new NullPointerException("exchange is null");
+        }
+
+        ReactiveStreamsConsumer target;
+        synchronized (this) {
+            requested--;
+            target = this.consumer;
+            if (target != null) {
+                inflightCount++;
+            }
+        }
+
+        if (target != null) {
+            target.process(exchange, doneSync -> {
+                synchronized (this) {
+                    inflightCount--;
+                }
+
+                refill();
+            });
+        } else {
+            // This may happen when the consumer is stopped
+            LOG.warn("Message received in stream '{}', but no consumers were attached. Discarding {}.", name, exchange);
+        }
+    }
+
+    protected void refill() {
+        Long toBeRequested = null;
+        Subscription subs = null;
+        synchronized (this) {
+            if (consumer != null && this.subscription != null) {
+                Integer consMax = consumer.getEndpoint().getMaxInflightExchanges();
+                long max = (consMax != null && consMax > 0) ? consMax.longValue() : MAX_INFLIGHT_UNBOUNDED;
+                long newRequest = max - requested - inflightCount;
+                if (newRequest > 0) {
+                    toBeRequested = newRequest;
+                    requested += toBeRequested;
+                    subs = this.subscription;
+                }
+            }
+        }
+
+        if (toBeRequested != null) {
+            subs.request(toBeRequested);
+        }
+    }
+
+    @Override
+    public void onError(Throwable throwable) {
+        if (throwable == null) {
+            throw new NullPointerException("throwable is null");
+        }
+
+        LOG.error("Error in reactive stream '" + name + "'", throwable);
+        synchronized (this) {
+            this.subscription = null;
+        }
+    }
+
+    @Override
+    public void onComplete() {
+        LOG.info("Reactive stream '{}' completed", name);
+        synchronized (this) {
+            this.subscription = null;
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        Subscription subscription;
+        synchronized (this) {
+            subscription = this.subscription;
+        }
+
+        if (subscription != null) {
+            subscription.cancel();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscription.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscription.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscription.java
new file mode 100644
index 0000000..e3489ef
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/CamelSubscription.java
@@ -0,0 +1,252 @@
+/**
+ * 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.reactive.streams.engine;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.component.reactive.streams.ReactiveStreamsBackpressureStrategy;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents a contract between a Camel published and an external subscriber.
+ * It manages backpressure in order to deal with slow subscribers.
+ */
+public class CamelSubscription implements Subscription {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CamelSubscription.class);
+
+    private ExecutorService workerPool;
+
+    private CamelPublisher publisher;
+
+    private ReactiveStreamsBackpressureStrategy backpressureStrategy;
+
+    private Subscriber<? super StreamPayload<Exchange>> subscriber;
+
+    /**
+     * The lock is used just for the time necessary to read/write shared variables.
+     */
+    private Lock mutex = new ReentrantLock(true);
+
+    private LinkedList<StreamPayload<Exchange>> buffer = new LinkedList<>();
+
+    /**
+     * The current number of exchanges requested by the subscriber.
+     */
+    private long requested;
+
+    /**
+     * Indicates that a cancel operation is to be performed.
+     */
+    private boolean terminating;
+
+    /**
+     * Indicates that the subscription is end.
+     */
+    private boolean terminated;
+
+    /**
+     * Indicates that a thread is currently sending items downstream.
+     * Items must be sent downstream by a single thread for each subscription.
+     */
+    private boolean sending;
+
+
+    public CamelSubscription(ExecutorService workerPool, CamelPublisher publisher, ReactiveStreamsBackpressureStrategy backpressureStrategy, Subscriber<? super StreamPayload<Exchange>> subscriber) {
+        this.workerPool = workerPool;
+        this.publisher = publisher;
+        this.backpressureStrategy = backpressureStrategy;
+        this.subscriber = subscriber;
+    }
+
+    @Override
+    public void request(long l) {
+        LOG.debug("Requested {} events from subscriber", l);
+        if (l <= 0) {
+            // wrong argument
+            mutex.lock();
+            terminated = true;
+            mutex.unlock();
+
+            publisher.unsubscribe(this);
+            subscriber.onError(new IllegalArgumentException("3.9"));
+        } else {
+            mutex.lock();
+            requested += l;
+            mutex.unlock();
+
+            checkAndFlush();
+        }
+    }
+
+    protected void checkAndFlush() {
+        mutex.lock();
+        boolean shouldFlush = !terminated && !sending && requested > 0 && buffer.size() > 0;
+        if (shouldFlush) {
+            sending = true;
+        }
+        mutex.unlock();
+
+        if (shouldFlush) {
+            workerPool.execute(() -> {
+
+                this.flush();
+
+                mutex.lock();
+                sending = false;
+                mutex.unlock();
+
+                // try again to flush
+                checkAndFlush();
+            });
+        } else {
+            mutex.lock();
+            boolean shouldComplete = terminating && !terminated;
+            if (shouldComplete) {
+                terminated = true;
+            }
+            mutex.unlock();
+
+            if (shouldComplete) {
+                this.publisher.unsubscribe(this);
+                this.subscriber.onComplete();
+                discardBuffer(this.buffer);
+            }
+        }
+    }
+
+    protected void flush() {
+        LinkedList<StreamPayload<Exchange>> sendingQueue = null;
+        try {
+            mutex.lock();
+
+            if (this.terminated) {
+                return;
+            }
+
+            int amount = (int) Math.min(requested, (long) buffer.size());
+            if (amount > 0) {
+                this.requested -= amount;
+                sendingQueue = new LinkedList<>();
+                while (amount > 0) {
+                    sendingQueue.add(buffer.removeFirst());
+                    amount--;
+                }
+            }
+
+        } finally {
+            mutex.unlock();
+        }
+
+        if (sendingQueue != null) {
+            LOG.debug("Sending {} events to the subscriber", sendingQueue.size());
+            for (StreamPayload<Exchange> data : sendingQueue) {
+                // TODO what if the subscriber throws an exception?
+                this.subscriber.onNext(data);
+
+                mutex.lock();
+                boolean shouldStop = this.terminated;
+                mutex.unlock();
+
+                if (shouldStop) {
+                    break;
+                }
+            }
+        }
+    }
+
+    public void signalCompletion() throws Exception {
+        mutex.lock();
+        terminating = true;
+        mutex.unlock();
+
+        checkAndFlush();
+    }
+
+    @Override
+    public void cancel() {
+        publisher.unsubscribe(this);
+
+        mutex.lock();
+        this.terminated = true;
+        List<StreamPayload<Exchange>> bufferCopy = new LinkedList<>(buffer);
+        this.buffer.clear();
+        mutex.unlock();
+
+        discardBuffer(bufferCopy);
+    }
+
+    protected void discardBuffer(List<StreamPayload<Exchange>> remaining) {
+        for (StreamPayload<Exchange> data : remaining) {
+            data.getCallback().processed(data.getItem(), new IllegalStateException("Cannot process the exchange " + data.getItem() + ": subscription cancelled"));
+        }
+    }
+
+    public void publish(StreamPayload<Exchange> message) {
+        StreamPayload<Exchange> discardedMessage = null;
+        String discardReason = null;
+        try {
+            mutex.lock();
+            if (!this.terminating && !this.terminated) {
+                if (this.backpressureStrategy == ReactiveStreamsBackpressureStrategy.BUFFER) {
+                    buffer.addLast(message);
+                } else if (this.backpressureStrategy == ReactiveStreamsBackpressureStrategy.DROP) {
+                    if (buffer.size() > 0) {
+                        LOG.warn("Exchange " + message.getItem() + " dropped according to the backpressure strategy " + ReactiveStreamsBackpressureStrategy.DROP);
+                        discardedMessage = message;
+                        discardReason = "the backpressure strategy (DROP) does not allow buffering";
+                    } else {
+                        buffer.addLast(message);
+                    }
+                } else if (this.backpressureStrategy == ReactiveStreamsBackpressureStrategy.LATEST) {
+                    if (buffer.size() > 0) {
+                        StreamPayload<Exchange> older = buffer.removeFirst();
+                        LOG.warn("Exchange " + message.getItem() + " dropped according to the backpressure strategy " + ReactiveStreamsBackpressureStrategy.LATEST);
+                        discardedMessage = older;
+                        discardReason = "the backpressure strategy (LATEST) does not allow buffering";
+                    }
+                    buffer.addLast(message);
+                } else {
+                    throw new IllegalStateException("Unsupported backpressure strategy: " + this.backpressureStrategy);
+                }
+
+            } else {
+                discardedMessage = message;
+                discardReason = "subscription closed";
+            }
+        } finally {
+            mutex.unlock();
+        }
+
+        if (discardedMessage != null) {
+            // acknowledge
+            discardedMessage.getCallback().processed(message.getItem(), new IllegalStateException("Exchange discarded: " + discardReason));
+        }
+
+        checkAndFlush();
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/ReactiveStreamsEngineConfiguration.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/ReactiveStreamsEngineConfiguration.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/ReactiveStreamsEngineConfiguration.java
new file mode 100644
index 0000000..af0fbec
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/ReactiveStreamsEngineConfiguration.java
@@ -0,0 +1,65 @@
+/**
+ * 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.reactive.streams.engine;
+
+/**
+ * Configuration parameters for the Camel internal reactive-streams engine.
+ */
+public class ReactiveStreamsEngineConfiguration implements Cloneable {
+
+    private String threadPoolName = "CamelReactiveStreamsWorker";
+
+    private int threadPoolMinSize;
+
+    private int threadPoolMaxSize = 10;
+
+    public ReactiveStreamsEngineConfiguration() {
+    }
+
+    public String getThreadPoolName() {
+        return threadPoolName;
+    }
+
+    /**
+     * The name of the thread pool used by the reactive streams internal engine.
+     */
+    public void setThreadPoolName(String threadPoolName) {
+        this.threadPoolName = threadPoolName;
+    }
+
+    public int getThreadPoolMinSize() {
+        return threadPoolMinSize;
+    }
+
+    /**
+     * The minimum number of threads used by the reactive streams internal engine.
+     */
+    public void setThreadPoolMinSize(int threadPoolMinSize) {
+        this.threadPoolMinSize = threadPoolMinSize;
+    }
+
+    public int getThreadPoolMaxSize() {
+        return threadPoolMaxSize;
+    }
+
+    /**
+     * The maximum number of threads used by the reactive streams internal engine.
+     */
+    public void setThreadPoolMaxSize(int threadPoolMaxSize) {
+        this.threadPoolMaxSize = threadPoolMaxSize;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java
new file mode 100644
index 0000000..b592c40
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/StreamPayload.java
@@ -0,0 +1,52 @@
+/**
+ * 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.reactive.streams.engine;
+
+import org.apache.camel.component.reactive.streams.api.DispatchCallback;
+
+/**
+ * A helper object that wraps the emitted item and the corresponding dispatch callback.
+ */
+public class StreamPayload<D> {
+
+    private D item;
+
+    private DispatchCallback<D> callback;
+
+    public StreamPayload(D item, DispatchCallback<D> callback) {
+        this.item = item;
+        this.callback = callback;
+    }
+
+    public D getItem() {
+        return item;
+    }
+
+    public DispatchCallback<D> getCallback() {
+        return callback;
+    }
+
+
+    @Override
+    public String toString() {
+        final StringBuilder sb = new StringBuilder("StreamPayload{");
+        sb.append("item=").append(item);
+        sb.append(", callback=").append(callback);
+        sb.append('}');
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java
new file mode 100644
index 0000000..1cf73cc
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/engine/UnwrappingPublisher.java
@@ -0,0 +1,87 @@
+/**
+ * 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.reactive.streams.engine;
+
+import java.util.Objects;
+
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A publisher that extracts the item from the payload as soon as it is delivered to the subscriber.
+ * It calls the dispatch callback if defined.
+ */
+public class UnwrappingPublisher<R> implements Publisher<R> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(UnwrappingPublisher.class);
+
+    private Publisher<StreamPayload<R>> delegate;
+
+    public UnwrappingPublisher(Publisher<StreamPayload<R>> delegate) {
+        Objects.requireNonNull(delegate, "delegate publisher cannot be null");
+        this.delegate = delegate;
+    }
+
+    @Override
+    public void subscribe(Subscriber<? super R> subscriber) {
+        delegate.subscribe(new Subscriber<StreamPayload<R>>() {
+
+            private Subscription subscription;
+
+            @Override
+            public void onSubscribe(Subscription subscription) {
+                if (subscription == null) {
+                    throw new NullPointerException("subscription is null");
+                }
+
+                if (this.subscription != null) {
+                    subscription.cancel();
+                } else {
+                    this.subscription = subscription;
+                    subscriber.onSubscribe(subscription);
+                }
+            }
+
+            @Override
+            public void onNext(StreamPayload<R> payload) {
+                Throwable error = null;
+                try {
+                    subscriber.onNext(payload.getItem());
+                } catch (Throwable t) {
+                    error = t;
+                }
+
+                if (payload.getCallback() != null) {
+                    payload.getCallback().processed(payload.getItem(), error);
+                }
+            }
+
+            @Override
+            public void onError(Throwable throwable) {
+                subscriber.onError(throwable);
+            }
+
+            @Override
+            public void onComplete() {
+                subscriber.onComplete();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java
new file mode 100644
index 0000000..e8c10dd
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingPublisher.java
@@ -0,0 +1,113 @@
+/**
+ * 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.reactive.streams.util;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.TypeConversionException;
+import org.reactivestreams.Publisher;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A publisher that converts Camel {@code Exchange}s into the given type.
+ */
+public class ConvertingPublisher<R> implements Publisher<R> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ConvertingPublisher.class);
+
+    private Publisher<Exchange> delegate;
+
+    private Class<R> type;
+
+    public ConvertingPublisher(Publisher<Exchange> delegate, Class<R> type) {
+        Objects.requireNonNull(delegate, "delegate publisher cannot be null");
+        this.delegate = delegate;
+        this.type = type;
+    }
+
+    @Override
+    public void subscribe(Subscriber<? super R> subscriber) {
+        delegate.subscribe(new Subscriber<Exchange>() {
+
+            private AtomicBoolean active = new AtomicBoolean(true);
+
+            private Subscription subscription;
+
+            @Override
+            public void onSubscribe(Subscription subscription) {
+                if (subscription == null) {
+                    throw new NullPointerException("subscription is null");
+                }
+                if (this.subscription != null) {
+                    subscription.cancel();
+                } else {
+                    this.subscription = subscription;
+                    subscriber.onSubscribe(subscription);
+                }
+            }
+
+            @Override
+            public void onNext(Exchange ex) {
+                if (!active.get()) {
+                    return;
+                }
+
+                R r;
+                try {
+                    r = ex.getIn().getBody(type);
+                } catch (TypeConversionException e) {
+                    LOG.warn("Unable to convert body to the specified type: " + type.getName(), e);
+                    r = null;
+                }
+
+                if (r == null && ex.getIn().getBody() != null) {
+                    this.onError(new ClassCastException("Unable to convert body to the specified type: " + type.getName()));
+
+                    active.set(false);
+                    subscription.cancel();
+                } else {
+                    subscriber.onNext(r);
+                }
+
+
+            }
+
+            @Override
+            public void onError(Throwable throwable) {
+                if (!active.get()) {
+                    return;
+                }
+
+                subscriber.onError(throwable);
+            }
+
+            @Override
+            public void onComplete() {
+                if (!active.get()) {
+                    return;
+                }
+
+                subscriber.onComplete();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/e66207a9/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingSubscriber.java
----------------------------------------------------------------------
diff --git a/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingSubscriber.java b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingSubscriber.java
new file mode 100644
index 0000000..cf4d4f0
--- /dev/null
+++ b/components/camel-reactive-streams/src/main/java/org/apache/camel/component/reactive/streams/util/ConvertingSubscriber.java
@@ -0,0 +1,75 @@
+/**
+ * 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.reactive.streams.util;
+
+import java.util.Objects;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.impl.DefaultExchange;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+/**
+ * A subscriber that converts items of the given type into Camel {@code Exchange}s.
+ */
+public class ConvertingSubscriber<R> implements Subscriber<R> {
+
+    private Subscriber<Exchange> delegate;
+
+    private CamelContext context;
+
+    public ConvertingSubscriber(Subscriber<Exchange> delegate, CamelContext context) {
+        Objects.requireNonNull(delegate, "delegate subscriber cannot be null");
+        this.delegate = delegate;
+        this.context = context;
+    }
+
+    @Override
+    public void onSubscribe(Subscription subscription) {
+        if (subscription == null) {
+            throw new NullPointerException("subscription is null");
+        }
+
+        delegate.onSubscribe(subscription);
+    }
+
+    @Override
+    public void onNext(R r) {
+        if (r == null) {
+            throw new NullPointerException("element is null");
+        }
+
+        Exchange exchange = new DefaultExchange(context);
+        exchange.getIn().setBody(r);
+        delegate.onNext(exchange);
+    }
+
+    @Override
+    public void onError(Throwable throwable) {
+        if (throwable == null) {
+            throw new NullPointerException("throwable is null");
+        }
+
+        delegate.onError(throwable);
+    }
+
+    @Override
+    public void onComplete() {
+        delegate.onComplete();
+    }
+}