You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2021/02/06 12:05:00 UTC

[camel] 07/18: CAMEL-16102: Source code generate @InvokeOnHeader for reflection free

This is an automated email from the ASF dual-hosted git repository.

davsclaus pushed a commit to branch on-header
in repository https://gitbox.apache.org/repos/asf/camel.git

commit 63c4df47ae1908e52e091643d32cd46a45aee652
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Thu Feb 4 16:04:00 2021 +0100

    CAMEL-16102: Source code generate @InvokeOnHeader for reflection free
---
 .../java/org/apache/camel/spi/InvokeOnHeader.java  |  17 +--
 .../impl/engine/DefaultUriFactoryResolver.java     |   7 +-
 .../apache/camel/support/BaseSelectorProducer.java |  56 ----------
 .../camel/support/HeaderSelectorProducer.java      | 118 ++++++++++-----------
 4 files changed, 70 insertions(+), 128 deletions(-)

diff --git a/core/camel-api/src/main/java/org/apache/camel/spi/InvokeOnHeader.java b/core/camel-api/src/main/java/org/apache/camel/spi/InvokeOnHeader.java
index fc0e977..0dc11e9 100644
--- a/core/camel-api/src/main/java/org/apache/camel/spi/InvokeOnHeader.java
+++ b/core/camel-api/src/main/java/org/apache/camel/spi/InvokeOnHeader.java
@@ -27,15 +27,18 @@ import org.apache.camel.Message;
 /**
  * Marks a method as being invoked for a specific header value.
  * <p/>
- * The method must have either of the following  method signatures:
+ * The method must have either of the following method signatures:
+ * 
  * <pre>
- *     void theMethodName(Message message) throws Exception;
- *     Object theMethodName(Message message) throws Exception;
- *     boolean theMethodName(Message message, AsyncCallback callback) throws Exception;
+ * void theMethodName(Message message) throws Exception;
+ * 
+ * Object theMethodName(Message message) throws Exception;
+ * 
+ * boolean theMethodName(Message message, AsyncCallback callback) throws Exception;
  * </pre>
- * If the method includes the {@link AsyncCallback} type, then the return value must be boolean,
- * as part of the async callback contract.
- * Throwing exceptions is optional and can be omitted.
+ * 
+ * If the method includes the {@link AsyncCallback} type, then the return value must be boolean, as part of the async
+ * callback contract. Throwing exceptions is optional and can be omitted.
  * <p/>
  * This can be used by Component implementations that uses org.apache.camel.support.HeaderSelectorProducer.
  *
diff --git a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultUriFactoryResolver.java b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultUriFactoryResolver.java
index acfb865..0558608 100644
--- a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultUriFactoryResolver.java
+++ b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultUriFactoryResolver.java
@@ -31,10 +31,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Default assembler resolver that looks for assembler factories in
+ * Default assembler resolver that looks for {@link UriFactoryResolver} factories in
  * <b>META-INF/services/org/apache/camel/urifactory/</b>.
  */
-
 public class DefaultUriFactoryResolver implements CamelContextAware, UriFactoryResolver {
     public static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/urifactory/";
 
@@ -70,7 +69,7 @@ public class DefaultUriFactoryResolver implements CamelContextAware, UriFactoryR
         // not in registry then use assembler factory for endpoints
         Class<?> type;
         try {
-            type = findAssembler(name + "-endpoint", context);
+            type = findFactory(name + "-endpoint", context);
         } catch (NoFactoryAvailableException e) {
             // its optional so its okay
             type = null;
@@ -99,7 +98,7 @@ public class DefaultUriFactoryResolver implements CamelContextAware, UriFactoryR
         return answer;
     }
 
-    private Class<?> findAssembler(String name, CamelContext context) throws IOException {
+    private Class<?> findFactory(String name, CamelContext context) throws IOException {
         if (factoryFinder == null) {
             factoryFinder = context.adapt(ExtendedCamelContext.class).getFactoryFinder(RESOURCE_PATH);
         }
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/BaseSelectorProducer.java b/core/camel-support/src/main/java/org/apache/camel/support/BaseSelectorProducer.java
deleted file mode 100644
index 8ab96b2..0000000
--- a/core/camel-support/src/main/java/org/apache/camel/support/BaseSelectorProducer.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.support;
-
-import org.apache.camel.Endpoint;
-import org.apache.camel.Exchange;
-import org.apache.camel.Processor;
-
-/**
- * A base class for selector-based producers.
- */
-public abstract class BaseSelectorProducer extends DefaultProducer {
-
-    protected BaseSelectorProducer(Endpoint endpoint) {
-        super(endpoint);
-    }
-
-    @Override
-    public void process(Exchange exchange) throws Exception {
-        final Processor processor = getProcessor(exchange);
-        if (processor != null) {
-            processor.process(exchange);
-        } else {
-            onMissingProcessor(exchange);
-        }
-    }
-
-    /**
-     * Determine the processor to use to handle the exchange.
-     *
-     * @param  exchange the message exchange
-     * @return          the processor to processes the message exchange
-     */
-    protected abstract Processor getProcessor(Exchange exchange) throws Exception;
-
-    /**
-     * Invoked when no processor has been defined to process the message exchange.
-     *
-     * @param exchange the message exchange
-     */
-    protected abstract void onMissingProcessor(Exchange exchange) throws Exception;
-}
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/HeaderSelectorProducer.java b/core/camel-support/src/main/java/org/apache/camel/support/HeaderSelectorProducer.java
index fbcb0c8..1aa2b49 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/HeaderSelectorProducer.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/HeaderSelectorProducer.java
@@ -16,35 +16,34 @@
  */
 package org.apache.camel.support;
 
-import java.lang.reflect.Method;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TreeMap;
 import java.util.function.Supplier;
 
+import org.apache.camel.AsyncCallback;
+import org.apache.camel.CamelContext;
+import org.apache.camel.CamelContextAware;
 import org.apache.camel.Endpoint;
 import org.apache.camel.Exchange;
-import org.apache.camel.Message;
+import org.apache.camel.ExtendedCamelContext;
 import org.apache.camel.NoSuchHeaderException;
-import org.apache.camel.Processor;
-import org.apache.camel.spi.InvokeOnHeader;
+import org.apache.camel.spi.InvokeOnHeaderStrategy;
 import org.apache.camel.util.ObjectHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.camel.support.ObjectHelper.invokeMethodSafe;
-
 /**
  * A selector-based producer which uses a header value to determine which processor should be invoked.
  */
-public abstract class HeaderSelectorProducer extends BaseSelectorProducer {
+public abstract class HeaderSelectorProducer extends DefaultAsyncProducer implements CamelContextAware {
+
+    public static final String RESOURCE_PATH = "META-INF/services/org/apache/camel/invoke-on-header/";
+
     private static final Logger LOGGER = LoggerFactory.getLogger(HeaderSelectorProducer.class);
 
     private final Supplier<String> headerSupplier;
     private final Supplier<String> defaultHeaderValueSupplier;
     private final Object target;
-    private Map<String, Processor> handlers;
+    private CamelContext camelContext;
+    private InvokeOnHeaderStrategy strategy;
 
     public HeaderSelectorProducer(Endpoint endpoint, Supplier<String> headerSupplier) {
         this(endpoint, headerSupplier, () -> null, null);
@@ -126,71 +125,68 @@ public abstract class HeaderSelectorProducer extends BaseSelectorProducer {
         this.headerSupplier = ObjectHelper.notNull(headerSupplier, "headerSupplier");
         this.defaultHeaderValueSupplier = ObjectHelper.notNull(defaultHeaderValueSupplier, "defaultHeaderValueSupplier");
         this.target = target != null ? target : this;
-        this.handlers = caseSensitive ? new HashMap<>() : new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
     }
 
     @Override
-    protected void doStart() throws Exception {
-        bind();
-
-        handlers = Collections.unmodifiableMap(handlers);
-
-        super.doStart();
+    public CamelContext getCamelContext() {
+        return camelContext;
     }
 
     @Override
-    protected Processor getProcessor(Exchange exchange) throws Exception {
-        String header = headerSupplier.get();
-        String action = exchange.getIn().getHeader(header, String.class);
-
-        if (action == null) {
-            action = defaultHeaderValueSupplier.get();
-        }
-        if (action == null) {
-            throw new NoSuchHeaderException(exchange, header, String.class);
-        }
-
-        return handlers.get(action);
+    public void setCamelContext(CamelContext camelContext) {
+        this.camelContext = camelContext;
     }
 
     @Override
-    protected void onMissingProcessor(Exchange exchange) throws Exception {
-        throw new IllegalStateException(
-                "Unsupported operation " + exchange.getIn().getHeader(headerSupplier.get()));
-    }
+    protected void doBuild() throws Exception {
+        super.doBuild();
 
-    // TODO: bind should use factory finder and use reflection free
-    // TODO: And setup this as part of doBuild as its loaded via classpath so we can build time optimize this
-    protected void bind() {
-        for (final Method method : getTarget().getClass().getDeclaredMethods()) {
-            bind(method.getAnnotation(InvokeOnHeader.class), method);
-        }
-    }
-
-    protected final void bind(String key, Processor processor) {
-        if (handlers.containsKey(key)) {
-            LOGGER.warn("A processor is already set for action {}", key);
-        }
-
-        this.handlers.put(key, processor);
+        String key = this.getClass().getName();
+        String fqn = RESOURCE_PATH + "/" + key;
+        strategy = camelContext.adapt(ExtendedCamelContext.class).getBootstrapFactoryFinder(RESOURCE_PATH)
+                .newInstance(key, InvokeOnHeaderStrategy.class)
+                .orElseThrow(() -> new IllegalArgumentException("Cannot find " + fqn + " in classpath."));
     }
 
-    protected void bind(InvokeOnHeader handler, final Method method) {
-        if (handler != null && method.getParameterCount() == 1) {
-            final Class<?> type = method.getParameterTypes()[0];
+    @Override
+    public boolean process(Exchange exchange, AsyncCallback callback) {
+        try {
+            String header = headerSupplier.get();
+            String action = exchange.getIn().getHeader(header, String.class);
 
-            LOGGER.debug("bind key={}, class={}, method={}, type={}",
-                    handler.value(), this.getClass(), method.getName(), type);
+            if (action == null) {
+                action = defaultHeaderValueSupplier.get();
+            }
+            if (action == null) {
+                throw new NoSuchHeaderException(exchange, header, String.class);
+            }
 
-            if (Message.class.isAssignableFrom(type)) {
-                bind(handler.value(), e -> invokeMethodSafe(method, target, e.getIn()));
-            } else {
-                bind(handler.value(), e -> invokeMethodSafe(method, target, e));
+            LOGGER.debug("Invoking @InvokeOnHeader method: {}", action);
+            Object answer = strategy.invoke(this, action, exchange, callback);
+            LOGGER.trace("Invoked @InvokeOnHeader method: {} -> {}", action, answer);
+
+            if (answer == null) {
+                // strategy invoked synchronously so trigger callback and return true
+                callback.done(true);
+                return true;
+            } else if (answer instanceof Boolean) {
+                boolean bool = (boolean) answer;
+                if (bool) {
+                    // strategy invoked synchronously so trigger callback and return true
+                    callback.done(true);
+                    return true;
+                } else {
+                    // strategy is invoking this asynchronously so return false
+                    return false;
+                }
             }
+        } catch (Exception e) {
+            exchange.setException(e);
+            callback.done(true);
+            return true;
         }
-    }
 
-    protected final Object getTarget() {
-        return this;
+        return false;
     }
+
 }