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 2020/10/23 12:17:56 UTC

[camel] 05/05: CAMEL-15732: Untangle reifier from builder.

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

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

commit 563f072d19eb3b40d67d022be45da6745a760380
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Fri Oct 23 14:12:33 2020 +0200

    CAMEL-15732: Untangle reifier from builder.
---
 .../org/apache/camel/jta/JtaTransactionPolicy.java |  10 +-
 .../camel/spring/spi/SpringTransactionPolicy.java  |  10 +-
 .../model/errorhandler/ErrorHandlerHelper.java     | 117 +++++++++++++++++++++
 .../errorhandler/ErrorHandlerRefReifier.java       |   3 +-
 .../reifier/errorhandler/ErrorHandlerReifier.java  |  97 -----------------
 .../DefaultManagementObjectNameStrategy.java       |  18 ++--
 6 files changed, 140 insertions(+), 115 deletions(-)

diff --git a/components/camel-jta/src/main/java/org/apache/camel/jta/JtaTransactionPolicy.java b/components/camel-jta/src/main/java/org/apache/camel/jta/JtaTransactionPolicy.java
index d9d159f..caf5c05 100644
--- a/components/camel-jta/src/main/java/org/apache/camel/jta/JtaTransactionPolicy.java
+++ b/components/camel-jta/src/main/java/org/apache/camel/jta/JtaTransactionPolicy.java
@@ -22,9 +22,10 @@ import org.apache.camel.Route;
 import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.builder.ErrorHandlerBuilderRef;
+import org.apache.camel.model.ModelCamelContext;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.model.RouteDefinition;
-import org.apache.camel.reifier.errorhandler.ErrorHandlerReifier;
+import org.apache.camel.model.errorhandler.ErrorHandlerHelper;
 import org.apache.camel.spi.TransactedPolicy;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -78,9 +79,9 @@ public abstract class JtaTransactionPolicy implements TransactedPolicy {
             // only lookup if there was explicit an error handler builder configured
             // otherwise its just the "default" that has not explicit been configured
             // and if so then we can safely replace that with our transacted error handler
-            if (ErrorHandlerReifier.isErrorHandlerFactoryConfigured(ref)) {
+            if (ErrorHandlerHelper.isErrorHandlerFactoryConfigured(ref)) {
                 LOG.debug("Looking up ErrorHandlerBuilder with ref: {}", ref);
-                builder = (ErrorHandlerBuilder) ErrorHandlerReifier.lookupErrorHandlerFactory(route, ref);
+                builder = (ErrorHandlerBuilder) ErrorHandlerHelper.lookupErrorHandlerFactory(route, ref, true);
             }
         }
 
@@ -121,7 +122,8 @@ public abstract class JtaTransactionPolicy implements TransactedPolicy {
             ErrorHandlerBuilder builder) {
         JtaTransactionErrorHandler answer;
         try {
-            answer = (JtaTransactionErrorHandler) ErrorHandlerReifier.reifier(route, builder).createErrorHandler(processor);
+            ModelCamelContext mcc = route.getCamelContext().adapt(ModelCamelContext.class);
+            answer = (JtaTransactionErrorHandler) mcc.getModelReifierFactory().createErrorHandler(route, builder, processor);
         } catch (Exception e) {
             throw RuntimeCamelException.wrapRuntimeCamelException(e);
         }
diff --git a/components/camel-spring/src/main/java/org/apache/camel/spring/spi/SpringTransactionPolicy.java b/components/camel-spring/src/main/java/org/apache/camel/spring/spi/SpringTransactionPolicy.java
index 6b895c4..d264ad4 100644
--- a/components/camel-spring/src/main/java/org/apache/camel/spring/spi/SpringTransactionPolicy.java
+++ b/components/camel-spring/src/main/java/org/apache/camel/spring/spi/SpringTransactionPolicy.java
@@ -22,8 +22,9 @@ import org.apache.camel.Route;
 import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.builder.ErrorHandlerBuilder;
 import org.apache.camel.builder.ErrorHandlerBuilderRef;
+import org.apache.camel.model.ModelCamelContext;
 import org.apache.camel.model.RouteDefinition;
-import org.apache.camel.reifier.errorhandler.ErrorHandlerReifier;
+import org.apache.camel.model.errorhandler.ErrorHandlerHelper;
 import org.apache.camel.spi.TransactedPolicy;
 import org.apache.camel.util.ObjectHelper;
 import org.slf4j.Logger;
@@ -80,9 +81,9 @@ public class SpringTransactionPolicy implements TransactedPolicy {
             // only lookup if there was explicit an error handler builder configured
             // otherwise its just the "default" that has not explicit been configured
             // and if so then we can safely replace that with our transacted error handler
-            if (ErrorHandlerReifier.isErrorHandlerFactoryConfigured(ref)) {
+            if (ErrorHandlerHelper.isErrorHandlerFactoryConfigured(ref)) {
                 LOG.debug("Looking up ErrorHandlerBuilder with ref: {}", ref);
-                builder = (ErrorHandlerBuilder) ErrorHandlerReifier.lookupErrorHandlerFactory(route, ref);
+                builder = (ErrorHandlerBuilder) ErrorHandlerHelper.lookupErrorHandlerFactory(route, ref, true);
             }
         }
 
@@ -120,7 +121,8 @@ public class SpringTransactionPolicy implements TransactedPolicy {
             Route route, Processor processor, ErrorHandlerBuilder builder) {
         TransactionErrorHandler answer;
         try {
-            answer = (TransactionErrorHandler) ErrorHandlerReifier.reifier(route, builder).createErrorHandler(processor);
+            ModelCamelContext mcc = route.getCamelContext().adapt(ModelCamelContext.class);
+            answer = (TransactionErrorHandler) mcc.getModelReifierFactory().createErrorHandler(route, builder, processor);
         } catch (Exception e) {
             throw RuntimeCamelException.wrapRuntimeCamelException(e);
         }
diff --git a/core/camel-core-engine/src/main/java/org/apache/camel/model/errorhandler/ErrorHandlerHelper.java b/core/camel-core-engine/src/main/java/org/apache/camel/model/errorhandler/ErrorHandlerHelper.java
new file mode 100644
index 0000000..f2f3ccf
--- /dev/null
+++ b/core/camel-core-engine/src/main/java/org/apache/camel/model/errorhandler/ErrorHandlerHelper.java
@@ -0,0 +1,117 @@
+/*
+ * 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.model.errorhandler;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.ErrorHandlerFactory;
+import org.apache.camel.ExtendedCamelContext;
+import org.apache.camel.Route;
+import org.apache.camel.model.ModelCamelContext;
+import org.apache.camel.support.CamelContextHelper;
+
+public final class ErrorHandlerHelper {
+
+    public static String DEFAULT_ERROR_HANDLER_BUILDER = "CamelDefaultErrorHandlerBuilder";
+
+    private ErrorHandlerHelper() {
+    }
+
+    /**
+     * Lookup the error handler by the given ref
+     *
+     * @param  route     the route
+     * @param  ref       reference id for the error handler
+     * @param  mandatory whether the error handler must exists, if not a {@link org.apache.camel.NoSuchBeanException} is
+     *                   thrown
+     * @return           the error handler
+     */
+    public static ErrorHandlerFactory lookupErrorHandlerFactory(Route route, String ref, boolean mandatory) {
+        ErrorHandlerFactory answer;
+        CamelContext camelContext = route.getCamelContext();
+
+        // if the ref is the default then we do not have any explicit error
+        // handler configured
+        // if that is the case then use error handlers configured on the route,
+        // as for instance
+        // the transacted error handler could have been configured on the route
+        // so we should use that one
+        if (!isErrorHandlerFactoryConfigured(ref)) {
+            // see if there has been configured a error handler builder on the route
+            answer = route.getErrorHandlerFactory();
+            // check if its also a ref with no error handler configuration like me
+            if (answer instanceof ErrorHandlerRefConfiguration) {
+                ErrorHandlerRefConfiguration other = (ErrorHandlerRefConfiguration) answer;
+                String otherRef = other.getRef();
+                if (!isErrorHandlerFactoryConfigured(otherRef)) {
+                    // the other has also no explicit error handler configured
+                    // then fallback to the handler
+                    // configured on the parent camel context
+                    answer = lookupErrorHandlerFactory(camelContext);
+                }
+                if (answer == null) {
+                    // the other has also no explicit error handler configured
+                    // then fallback to the default error handler
+                    // otherwise we could recursive loop forever (triggered by
+                    // createErrorHandler method)
+                    answer = camelContext.adapt(ModelCamelContext.class).getModelReifierFactory().createDefaultErrorHandler();
+                }
+                // inherit the error handlers from the other as they are to be
+                // shared
+                // this is needed by camel-spring when none error handler has
+                // been explicit configured
+                route.addErrorHandlerFactoryReference(other, answer);
+            }
+        } else {
+            // use specific configured error handler
+            if (mandatory) {
+                answer = CamelContextHelper.mandatoryLookup(camelContext, ref, ErrorHandlerFactory.class);
+            } else {
+                answer = CamelContextHelper.lookup(camelContext, ref, ErrorHandlerFactory.class);
+            }
+        }
+
+        return answer;
+    }
+
+    protected static ErrorHandlerFactory lookupErrorHandlerFactory(CamelContext camelContext) {
+        ErrorHandlerFactory answer = camelContext.adapt(ExtendedCamelContext.class).getErrorHandlerFactory();
+        if (answer instanceof ErrorHandlerRefConfiguration) {
+            ErrorHandlerRefConfiguration other = (ErrorHandlerRefConfiguration) answer;
+            String otherRef = other.getRef();
+            if (isErrorHandlerFactoryConfigured(otherRef)) {
+                answer = CamelContextHelper.lookup(camelContext, otherRef, ErrorHandlerFactory.class);
+                if (answer == null) {
+                    throw new IllegalArgumentException("ErrorHandlerBuilder with id " + otherRef + " not found in registry.");
+                }
+            }
+        }
+
+        return answer;
+    }
+
+    /**
+     * Returns whether a specific error handler builder has been configured or not.
+     * <p/>
+     * Can be used to test if none has been configured and then install a custom error handler builder replacing the
+     * default error handler (that would have been used as fallback otherwise). <br/>
+     * This is for instance used by the transacted policy to setup a TransactedErrorHandlerBuilder in camel-spring.
+     */
+    public static boolean isErrorHandlerFactoryConfigured(String ref) {
+        return !ErrorHandlerRefConfiguration.DEFAULT_ERROR_HANDLER_BUILDER.equals(ref);
+    }
+
+}
diff --git a/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerRefReifier.java b/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerRefReifier.java
index 120814b..547f712 100644
--- a/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerRefReifier.java
+++ b/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerRefReifier.java
@@ -20,6 +20,7 @@ import org.apache.camel.ErrorHandlerFactory;
 import org.apache.camel.Processor;
 import org.apache.camel.Route;
 import org.apache.camel.model.ModelCamelContext;
+import org.apache.camel.model.errorhandler.ErrorHandlerHelper;
 import org.apache.camel.model.errorhandler.ErrorHandlerRefConfiguration;
 import org.apache.camel.util.ObjectHelper;
 
@@ -37,7 +38,7 @@ public class ErrorHandlerRefReifier extends ErrorHandlerReifier<ErrorHandlerRefC
     }
 
     private ErrorHandlerFactory lookupErrorHandler(Route route) {
-        ErrorHandlerFactory handler = ErrorHandlerReifier.lookupErrorHandlerFactory(route, definition.getRef());
+        ErrorHandlerFactory handler = ErrorHandlerHelper.lookupErrorHandlerFactory(route, definition.getRef(), true);
         ObjectHelper.notNull(handler, "error handler '" + definition.getRef() + "'");
         route.addErrorHandlerFactoryReference(definition, handler);
         return handler;
diff --git a/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerReifier.java b/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerReifier.java
index be10ddd..15dd813 100644
--- a/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerReifier.java
+++ b/core/camel-core-engine/src/main/java/org/apache/camel/reifier/errorhandler/ErrorHandlerReifier.java
@@ -25,14 +25,12 @@ import java.util.function.BiFunction;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.ErrorHandlerFactory;
-import org.apache.camel.ExtendedCamelContext;
 import org.apache.camel.LoggingLevel;
 import org.apache.camel.NamedNode;
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
 import org.apache.camel.Route;
 import org.apache.camel.RuntimeCamelException;
-import org.apache.camel.model.ModelCamelContext;
 import org.apache.camel.model.OnExceptionDefinition;
 import org.apache.camel.model.RedeliveryPolicyDefinition;
 import org.apache.camel.model.errorhandler.DeadLetterChannelConfiguration;
@@ -167,101 +165,6 @@ public abstract class ErrorHandlerReifier<T extends ErrorHandlerFactory> extends
         }
     }
 
-    /**
-     * Lookup the error handler by the given ref
-     *
-     * @param  route the route context
-     * @param  ref   reference id for the error handler
-     * @return       the error handler
-     */
-    public static ErrorHandlerFactory lookupErrorHandlerFactory(Route route, String ref) {
-        return lookupErrorHandlerFactory(route, ref, true);
-    }
-
-    /**
-     * Lookup the error handler by the given ref
-     *
-     * @param  route     the route
-     * @param  ref       reference id for the error handler
-     * @param  mandatory whether the error handler must exists, if not a {@link org.apache.camel.NoSuchBeanException} is
-     *                   thrown
-     * @return           the error handler
-     */
-    public static ErrorHandlerFactory lookupErrorHandlerFactory(Route route, String ref, boolean mandatory) {
-        ErrorHandlerFactory answer;
-        CamelContext camelContext = route.getCamelContext();
-
-        // if the ref is the default then we do not have any explicit error
-        // handler configured
-        // if that is the case then use error handlers configured on the route,
-        // as for instance
-        // the transacted error handler could have been configured on the route
-        // so we should use that one
-        if (!isErrorHandlerFactoryConfigured(ref)) {
-            // see if there has been configured a error handler builder on the route
-            answer = route.getErrorHandlerFactory();
-            // check if its also a ref with no error handler configuration like me
-            if (answer instanceof ErrorHandlerRefConfiguration) {
-                ErrorHandlerRefConfiguration other = (ErrorHandlerRefConfiguration) answer;
-                String otherRef = other.getRef();
-                if (!isErrorHandlerFactoryConfigured(otherRef)) {
-                    // the other has also no explicit error handler configured
-                    // then fallback to the handler
-                    // configured on the parent camel context
-                    answer = lookupErrorHandlerFactory(camelContext);
-                }
-                if (answer == null) {
-                    // the other has also no explicit error handler configured
-                    // then fallback to the default error handler
-                    // otherwise we could recursive loop forever (triggered by
-                    // createErrorHandler method)
-                    answer = camelContext.adapt(ModelCamelContext.class).getModelReifierFactory().createDefaultErrorHandler();
-                }
-                // inherit the error handlers from the other as they are to be
-                // shared
-                // this is needed by camel-spring when none error handler has
-                // been explicit configured
-                route.addErrorHandlerFactoryReference(other, answer);
-            }
-        } else {
-            // use specific configured error handler
-            if (mandatory) {
-                answer = CamelContextHelper.mandatoryLookup(camelContext, ref, ErrorHandlerFactory.class);
-            } else {
-                answer = CamelContextHelper.lookup(camelContext, ref, ErrorHandlerFactory.class);
-            }
-        }
-
-        return answer;
-    }
-
-    protected static ErrorHandlerFactory lookupErrorHandlerFactory(CamelContext camelContext) {
-        ErrorHandlerFactory answer = camelContext.adapt(ExtendedCamelContext.class).getErrorHandlerFactory();
-        if (answer instanceof ErrorHandlerRefConfiguration) {
-            ErrorHandlerRefConfiguration other = (ErrorHandlerRefConfiguration) answer;
-            String otherRef = other.getRef();
-            if (isErrorHandlerFactoryConfigured(otherRef)) {
-                answer = CamelContextHelper.lookup(camelContext, otherRef, ErrorHandlerFactory.class);
-                if (answer == null) {
-                    throw new IllegalArgumentException("ErrorHandlerBuilder with id " + otherRef + " not found in registry.");
-                }
-            }
-        }
-
-        return answer;
-    }
-
-    /**
-     * Returns whether a specific error handler builder has been configured or not.
-     * <p/>
-     * Can be used to test if none has been configured and then install a custom error handler builder replacing the
-     * default error handler (that would have been used as fallback otherwise). <br/>
-     * This is for instance used by the transacted policy to setup a TransactedErrorHandlerBuilder in camel-spring.
-     */
-    public static boolean isErrorHandlerFactoryConfigured(String ref) {
-        return !ErrorHandlerRefConfiguration.DEFAULT_ERROR_HANDLER_BUILDER.equals(ref);
-    }
-
     public void addExceptionPolicy(ErrorHandlerSupport handlerSupport, OnExceptionDefinition exceptionType) {
         // add error handler as child service so they get lifecycle handled
         Processor errorHandler = route.getOnException(exceptionType.getId());
diff --git a/core/camel-management/src/main/java/org/apache/camel/management/DefaultManagementObjectNameStrategy.java b/core/camel-management/src/main/java/org/apache/camel/management/DefaultManagementObjectNameStrategy.java
index b1e8213..02ad3da 100644
--- a/core/camel-management/src/main/java/org/apache/camel/management/DefaultManagementObjectNameStrategy.java
+++ b/core/camel-management/src/main/java/org/apache/camel/management/DefaultManagementObjectNameStrategy.java
@@ -34,7 +34,6 @@ import org.apache.camel.Producer;
 import org.apache.camel.Route;
 import org.apache.camel.Service;
 import org.apache.camel.StaticService;
-import org.apache.camel.builder.ErrorHandlerBuilderRef;
 import org.apache.camel.cluster.CamelClusterService;
 import org.apache.camel.management.mbean.ManagedBacklogDebugger;
 import org.apache.camel.management.mbean.ManagedBacklogTracer;
@@ -56,7 +55,8 @@ import org.apache.camel.management.mbean.ManagedStep;
 import org.apache.camel.management.mbean.ManagedSupervisingRouteController;
 import org.apache.camel.management.mbean.ManagedThreadPool;
 import org.apache.camel.management.mbean.ManagedTracer;
-import org.apache.camel.reifier.errorhandler.ErrorHandlerReifier;
+import org.apache.camel.model.errorhandler.ErrorHandlerHelper;
+import org.apache.camel.model.errorhandler.ErrorHandlerRefConfiguration;
 import org.apache.camel.spi.DataFormat;
 import org.apache.camel.spi.EventNotifier;
 import org.apache.camel.spi.ManagementObjectNameStrategy;
@@ -320,12 +320,12 @@ public class DefaultManagementObjectNameStrategy implements ManagementObjectName
         // we want to only register one instance of the various error handler types and thus do some lookup
         // if its a ErrorHandlerBuildRef. We need a bit of work to do that as there are potential indirection.
         String ref = null;
-        if (builder instanceof ErrorHandlerBuilderRef) {
-            ErrorHandlerBuilderRef builderRef = (ErrorHandlerBuilderRef) builder;
+        if (builder instanceof ErrorHandlerRefConfiguration) {
+            ErrorHandlerRefConfiguration builderRef = (ErrorHandlerRefConfiguration) builder;
 
             // it has not then its an indirection and we should do some work to lookup the real builder
             ref = builderRef.getRef();
-            ErrorHandlerFactory refBuilder = ErrorHandlerReifier.lookupErrorHandlerFactory(route, builderRef.getRef(), false);
+            ErrorHandlerFactory refBuilder = ErrorHandlerHelper.lookupErrorHandlerFactory(route, builderRef.getRef(), false);
             if (refBuilder != null) {
                 builder = refBuilder;
             }
@@ -333,11 +333,11 @@ public class DefaultManagementObjectNameStrategy implements ManagementObjectName
             // must do a 2nd lookup in case this is also a reference
             // (this happens with spring DSL using errorHandlerRef on <route> as it gets a bit
             // complex with indirections for error handler references
-            if (builder instanceof ErrorHandlerBuilderRef) {
-                builderRef = (ErrorHandlerBuilderRef) builder;
+            if (builder instanceof ErrorHandlerRefConfiguration) {
+                builderRef = (ErrorHandlerRefConfiguration) builder;
                 // does it refer to a non default error handler then do a 2nd lookup
-                if (!builderRef.getRef().equals(ErrorHandlerBuilderRef.DEFAULT_ERROR_HANDLER_BUILDER)) {
-                    refBuilder = ErrorHandlerReifier.lookupErrorHandlerFactory(route, builderRef.getRef(), false);
+                if (!builderRef.getRef().equals(ErrorHandlerRefConfiguration.DEFAULT_ERROR_HANDLER_BUILDER)) {
+                    refBuilder = ErrorHandlerHelper.lookupErrorHandlerFactory(route, builderRef.getRef(), false);
                     if (refBuilder != null) {
                         ref = builderRef.getRef();
                         builder = refBuilder;