orpiske commented on code in PR #11906: URL: https://github.com/apache/camel/pull/11906#discussion_r1383095951
########## components/camel-dynamic-router/src/main/java/org/apache/camel/component/dynamicrouter/DynamicRouterEndpoint.java: ########## @@ -140,30 +149,87 @@ protected void doInit() throws Exception { super.doInit(); DynamicRouterComponent component = getDynamicRouterComponent(); if (CONTROL_CHANNEL_NAME.equals(configuration.getChannel())) { - final DynamicRouterControlChannelProcessor processor = controlChannelProcessorFactorySupplier.get() - .getInstance(component); - processor.setConfiguration(configuration); try { // There can be multiple control actions, but we do not want to // create another consumer on the control channel, so check to // see if the consumer has already been created, and skip the // creation of another consumer if one already exists if (component.getControlChannelProcessor() == null) { + DynamicRouterControlChannelProcessor processor = controlChannelProcessorFactorySupplier.get() + .getInstance(component); + processor.setConfiguration(configuration); component.setControlChannelProcessor(processor); } } catch (Exception e) { throw new IllegalStateException("Could not create Dynamic Router endpoint", e); } } else { - final DynamicRouterProcessor processor = processorFactorySupplier.get() - .getInstance("dynamicRouterProcessor-" + configuration.getChannel(), getCamelContext(), - configuration.getRecipientMode(), configuration.isWarnDroppedMessage(), - filterProcessorFactorySupplier); - ServiceHelper.startService(processor); + CamelContext camelContext = getCamelContext(); + String routeId = configuration.getRouteId(); + long timeout = configuration.getTimeout(); + ErrorHandler errorHandler = new NoErrorHandler(null); + if (producerCache == null) { + producerCache = new DefaultProducerCache(this, camelContext, 1000); + } + ExecutorService aggregateExecutorService = camelContext.getExecutorServiceManager() + .newScheduledThreadPool(this, "DynamicRouter-AggregateTask", 0); + if (timeout > 0) { + // use a cached thread pool so we each on-the-fly task has a dedicated thread to process completions as they come in + aggregateExecutorService = camelContext.getExecutorServiceManager() + .newScheduledThreadPool(this, "DynamicRouter-AggregateTask", 0); + } + AggregationStrategy aggregationStrategy = determineAggregationStrategy(camelContext); + DynamicRouterMulticastProcessor processor = processorFactorySupplier.get() + .getInstance("DynamicRouterMulticastProcessor-" + configuration.getChannel(), camelContext, null, + configuration.getRecipientMode(), + configuration.isWarnDroppedMessage(), filterProcessorFactorySupplier, producerCache, + aggregationStrategy, configuration.isParallelProcessing(), + determineExecutorService(camelContext), configuration.isShutdownExecutorService(), + configuration.isStreaming(), configuration.isStopOnException(), timeout, + determineOnPrepare(camelContext), configuration.isShareUnitOfWork(), + configuration.isParallelAggregate()); + processor.setErrorHandler(errorHandler); + processor.setAggregateExecutorService(aggregateExecutorService); + processor.setIgnoreInvalidEndpoints(configuration.isIgnoreInvalidEndpoints()); + processor.setId(getId()); + processor.setRouteId(routeId); + ServiceHelper.startService(aggregationStrategy, producerCache, processor); component.addRoutingProcessor(configuration.getChannel(), processor); } } + protected ExecutorService determineExecutorService(CamelContext camelContext) { + ExecutorService executorService = null; + if (ObjectHelper.isNotEmpty(configuration.getExecutorService())) { + executorService = camelContext.getExecutorServiceManager() + .newThreadPool(this, "@RecipientList", configuration.getExecutorService()); + } + if (configuration.isParallelProcessing() && configuration.getExecutorService() == null) { + // we are running in parallel, so we need a thread pool + executorService = camelContext.getExecutorServiceManager() + .newDefaultThreadPool(this, "@RecipientList"); + } + return executorService; + } + + protected AggregationStrategy determineAggregationStrategy(CamelContext camelContext) { + AggregationStrategy aggregationStrategy = new UseLatestAggregationStrategy(); + if (ObjectHelper.isNotEmpty(configuration.getAggregationStrategy())) { + aggregationStrategy = CamelContextHelper.mandatoryLookup(camelContext, + configuration.getAggregationStrategy(), AggregationStrategy.class); + } + return aggregationStrategy; + } + + protected Processor determineOnPrepare(CamelContext camelContext) { + Processor processor = exchange -> { + }; + if (ObjectHelper.isNotEmpty(configuration.getOnPrepare())) { + processor = CamelContextHelper.mandatoryLookup(camelContext, configuration.getOnPrepare(), Processor.class); + } + return processor; Review Comment: Maybe return a static processor to avoid the lambda/instantiation overhead. Something like: ``` if (ObjectHelper.isNotEmpty(configuration.getOnPrepare())) { return CamelContextHelper.mandatoryLookup(camelContext, configuration.getOnPrepare(), Processor.class); } return SOME_STATIC_EMPTY_PROCESSOR; ``` ########## components/camel-dynamic-router/src/main/java/org/apache/camel/component/dynamicrouter/DynamicRouterEndpoint.java: ########## @@ -16,21 +16,26 @@ */ package org.apache.camel.component.dynamicrouter; +import java.util.concurrent.ExecutorService; import java.util.function.Supplier; -import org.apache.camel.Category; -import org.apache.camel.Consumer; -import org.apache.camel.Processor; -import org.apache.camel.Producer; +import org.apache.camel.*; Review Comment: Just a minor thing: we tend to avoid using star imports. ########## components/camel-dynamic-router/src/main/java/org/apache/camel/component/dynamicrouter/DynamicRouterEndpoint.java: ########## @@ -140,30 +149,87 @@ protected void doInit() throws Exception { super.doInit(); DynamicRouterComponent component = getDynamicRouterComponent(); if (CONTROL_CHANNEL_NAME.equals(configuration.getChannel())) { - final DynamicRouterControlChannelProcessor processor = controlChannelProcessorFactorySupplier.get() - .getInstance(component); - processor.setConfiguration(configuration); try { // There can be multiple control actions, but we do not want to // create another consumer on the control channel, so check to // see if the consumer has already been created, and skip the // creation of another consumer if one already exists if (component.getControlChannelProcessor() == null) { + DynamicRouterControlChannelProcessor processor = controlChannelProcessorFactorySupplier.get() + .getInstance(component); + processor.setConfiguration(configuration); component.setControlChannelProcessor(processor); } } catch (Exception e) { throw new IllegalStateException("Could not create Dynamic Router endpoint", e); } } else { - final DynamicRouterProcessor processor = processorFactorySupplier.get() - .getInstance("dynamicRouterProcessor-" + configuration.getChannel(), getCamelContext(), - configuration.getRecipientMode(), configuration.isWarnDroppedMessage(), - filterProcessorFactorySupplier); - ServiceHelper.startService(processor); + CamelContext camelContext = getCamelContext(); + String routeId = configuration.getRouteId(); + long timeout = configuration.getTimeout(); + ErrorHandler errorHandler = new NoErrorHandler(null); + if (producerCache == null) { + producerCache = new DefaultProducerCache(this, camelContext, 1000); + } + ExecutorService aggregateExecutorService = camelContext.getExecutorServiceManager() + .newScheduledThreadPool(this, "DynamicRouter-AggregateTask", 0); + if (timeout > 0) { + // use a cached thread pool so we each on-the-fly task has a dedicated thread to process completions as they come in + aggregateExecutorService = camelContext.getExecutorServiceManager() + .newScheduledThreadPool(this, "DynamicRouter-AggregateTask", 0); + } + AggregationStrategy aggregationStrategy = determineAggregationStrategy(camelContext); + DynamicRouterMulticastProcessor processor = processorFactorySupplier.get() + .getInstance("DynamicRouterMulticastProcessor-" + configuration.getChannel(), camelContext, null, + configuration.getRecipientMode(), + configuration.isWarnDroppedMessage(), filterProcessorFactorySupplier, producerCache, + aggregationStrategy, configuration.isParallelProcessing(), + determineExecutorService(camelContext), configuration.isShutdownExecutorService(), + configuration.isStreaming(), configuration.isStopOnException(), timeout, + determineOnPrepare(camelContext), configuration.isShareUnitOfWork(), + configuration.isParallelAggregate()); + processor.setErrorHandler(errorHandler); + processor.setAggregateExecutorService(aggregateExecutorService); + processor.setIgnoreInvalidEndpoints(configuration.isIgnoreInvalidEndpoints()); + processor.setId(getId()); + processor.setRouteId(routeId); Review Comment: I'd probably split this part outside of the init method. I've been broken these large methods in core for a while because: 1. they are hard to read and maintain and 2. they affect JIT compiler ability to inline code when needed. Even though this is not on the hot path, I would still like to keep the code easier to read/maintain. ########## components/camel-dynamic-router/src/main/java/org/apache/camel/component/dynamicrouter/DynamicRouterMulticastProcessor.java: ########## @@ -0,0 +1,292 @@ +/* + * 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.dynamicrouter; + +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import org.apache.camel.*; +import org.apache.camel.processor.FilterProcessor; +import org.apache.camel.processor.MulticastProcessor; +import org.apache.camel.processor.ProcessorExchangePair; +import org.apache.camel.spi.ProducerCache; +import org.apache.camel.support.*; +import org.apache.camel.support.service.ServiceHelper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.camel.component.dynamicrouter.DynamicRouterConstants.MODE_FIRST_MATCH; + +public class DynamicRouterMulticastProcessor extends MulticastProcessor { + + private static final Logger LOG = LoggerFactory.getLogger(DynamicRouterMulticastProcessor.class); + + /** + * Template for a logging endpoint, showing all, and multiline. + */ + private static final String LOG_ENDPOINT = "log:%s.%s?level=%s&showAll=true&multiline=true"; + + private boolean ignoreInvalidEndpoints; + + private final ProducerCache producerCache; + + /** + * {@link FilterProcessor}s, mapped by subscription ID, to determine if the incoming exchange should be routed based + * on the content. + */ + private final TreeMap<String, PrioritizedFilter> filterMap; + + /** + * Indicates the behavior of the Dynamic Router when routing participants are selected to receive an incoming + * exchange. If the mode is "firstMatch", then the exchange is routed only to the first participant that has a + * matching predicate. If the mode is "allMatch", then the exchange is routed to all participants that have a + * matching predicate. + */ + private final String recipientMode; + + /** + * The {@link FilterProcessor} factory. + */ + private final Supplier<PrioritizedFilter.PrioritizedFilterFactory> filterProcessorFactorySupplier; + + /** + * Flag to log a warning if a message is dropped due to no matching filters. + */ + private final boolean warnDroppedMessage; + + public DynamicRouterMulticastProcessor(String id, CamelContext camelContext, Route route, String recipientMode, + final boolean warnDroppedMessage, + final Supplier<PrioritizedFilter.PrioritizedFilterFactory> filterProcessorFactorySupplier, + ProducerCache producerCache, AggregationStrategy aggregationStrategy, + boolean parallelProcessing, ExecutorService executorService, + boolean shutdownExecutorService, + boolean streaming, boolean stopOnException, + long timeout, Processor onPrepare, boolean shareUnitOfWork, + boolean parallelAggregate) { + super(camelContext, route, new ArrayList<>(), aggregationStrategy, parallelProcessing, executorService, + shutdownExecutorService, + streaming, stopOnException, timeout, onPrepare, + shareUnitOfWork, parallelAggregate); + setId(id); + this.producerCache = producerCache; + this.filterMap = new TreeMap<>(); + this.recipientMode = recipientMode; + this.filterProcessorFactorySupplier = filterProcessorFactorySupplier; + this.warnDroppedMessage = warnDroppedMessage; + } + + public boolean isIgnoreInvalidEndpoints() { + return ignoreInvalidEndpoints; + } + + public void setIgnoreInvalidEndpoints(boolean ignoreInvalidEndpoints) { + this.ignoreInvalidEndpoints = ignoreInvalidEndpoints; + } + + protected List<Processor> createEndpointProcessors(Exchange exchange) { + List<String> recipientList = matchFilters(exchange).stream() + .map(PrioritizedFilter::getEndpoint) + .distinct() + .map(String::trim) + .collect(Collectors.toList()); + if (recipientList.isEmpty()) { + // No matching filters, so we will use the default filter that will create a + // notification that there were no routing participants that matched the + // exchange, which results in a "dropped" message. + Message exchangeIn = exchange.getIn(); + Object originalBody = exchangeIn.getBody(); + exchangeIn.setHeader("originalBody", originalBody); + String endpoint = String.format(LOG_ENDPOINT, this.getClass().getCanonicalName(), getId(), + warnDroppedMessage ? "WARN" : "DEBUG"); + recipientList.add(endpoint); + String error = String.format( + "DynamicRouter '%s': no filters matched for an exchange with id: '%s', from route: '%s'. " + + "The 'originalBody' header contains the original message body.", + getId(), exchange.getExchangeId(), exchange.getFromEndpoint()); + exchangeIn.setBody(error, String.class); + } + + return recipientList.stream() + .map(uri -> { + Endpoint endpoint; + try { + endpoint = Optional.ofNullable(exchange.getContext().hasEndpoint(uri)) + .orElse(ExchangeHelper.resolveEndpoint(exchange, uri)); + } catch (Exception e) { + if (isIgnoreInvalidEndpoints()) { + LOG.debug("Endpoint uri is invalid: {}. This exception will be ignored.", uri, e); + return null; + } else { + // failure so break out + throw e; + } + } + return endpoint; + }) + .filter(Objects::nonNull) + .map(endpoint -> { + Producer producer = producerCache.acquireProducer(endpoint); + Route route = ExchangeHelper.getRoute(exchange); + return wrapInErrorHandler(route, exchange, producer); + }) + .toList(); + } Review Comment: Re: my comment above about this being too large. I would also move this separately and, I think, I would reconsider using the streams here as it would create iterator objects under the hood and could add pressure to the GC. ########## components/camel-dynamic-router/src/main/java/org/apache/camel/component/dynamicrouter/DynamicRouterMulticastProcessor.java: ########## @@ -0,0 +1,292 @@ +/* + * 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.dynamicrouter; + +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import org.apache.camel.*; +import org.apache.camel.processor.FilterProcessor; +import org.apache.camel.processor.MulticastProcessor; +import org.apache.camel.processor.ProcessorExchangePair; +import org.apache.camel.spi.ProducerCache; +import org.apache.camel.support.*; +import org.apache.camel.support.service.ServiceHelper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.camel.component.dynamicrouter.DynamicRouterConstants.MODE_FIRST_MATCH; + +public class DynamicRouterMulticastProcessor extends MulticastProcessor { + + private static final Logger LOG = LoggerFactory.getLogger(DynamicRouterMulticastProcessor.class); + + /** + * Template for a logging endpoint, showing all, and multiline. + */ + private static final String LOG_ENDPOINT = "log:%s.%s?level=%s&showAll=true&multiline=true"; + + private boolean ignoreInvalidEndpoints; + + private final ProducerCache producerCache; + + /** + * {@link FilterProcessor}s, mapped by subscription ID, to determine if the incoming exchange should be routed based + * on the content. + */ + private final TreeMap<String, PrioritizedFilter> filterMap; + + /** + * Indicates the behavior of the Dynamic Router when routing participants are selected to receive an incoming + * exchange. If the mode is "firstMatch", then the exchange is routed only to the first participant that has a + * matching predicate. If the mode is "allMatch", then the exchange is routed to all participants that have a + * matching predicate. + */ + private final String recipientMode; + + /** + * The {@link FilterProcessor} factory. + */ + private final Supplier<PrioritizedFilter.PrioritizedFilterFactory> filterProcessorFactorySupplier; + + /** + * Flag to log a warning if a message is dropped due to no matching filters. + */ + private final boolean warnDroppedMessage; + + public DynamicRouterMulticastProcessor(String id, CamelContext camelContext, Route route, String recipientMode, + final boolean warnDroppedMessage, + final Supplier<PrioritizedFilter.PrioritizedFilterFactory> filterProcessorFactorySupplier, + ProducerCache producerCache, AggregationStrategy aggregationStrategy, + boolean parallelProcessing, ExecutorService executorService, + boolean shutdownExecutorService, + boolean streaming, boolean stopOnException, + long timeout, Processor onPrepare, boolean shareUnitOfWork, + boolean parallelAggregate) { + super(camelContext, route, new ArrayList<>(), aggregationStrategy, parallelProcessing, executorService, + shutdownExecutorService, + streaming, stopOnException, timeout, onPrepare, + shareUnitOfWork, parallelAggregate); + setId(id); + this.producerCache = producerCache; + this.filterMap = new TreeMap<>(); + this.recipientMode = recipientMode; + this.filterProcessorFactorySupplier = filterProcessorFactorySupplier; + this.warnDroppedMessage = warnDroppedMessage; + } + + public boolean isIgnoreInvalidEndpoints() { + return ignoreInvalidEndpoints; + } + + public void setIgnoreInvalidEndpoints(boolean ignoreInvalidEndpoints) { + this.ignoreInvalidEndpoints = ignoreInvalidEndpoints; + } + + protected List<Processor> createEndpointProcessors(Exchange exchange) { Review Comment: This method is very likely too large to be inlined by the JIT compiler. I haven't analyzed, but I suspect it could be on the hot path ... so, ideally, it should be broken in smaller pieces. ########## components/camel-dynamic-router/src/main/java/org/apache/camel/component/dynamicrouter/DynamicRouterEndpoint.java: ########## @@ -140,30 +149,87 @@ protected void doInit() throws Exception { super.doInit(); DynamicRouterComponent component = getDynamicRouterComponent(); if (CONTROL_CHANNEL_NAME.equals(configuration.getChannel())) { - final DynamicRouterControlChannelProcessor processor = controlChannelProcessorFactorySupplier.get() - .getInstance(component); - processor.setConfiguration(configuration); try { // There can be multiple control actions, but we do not want to // create another consumer on the control channel, so check to // see if the consumer has already been created, and skip the // creation of another consumer if one already exists if (component.getControlChannelProcessor() == null) { + DynamicRouterControlChannelProcessor processor = controlChannelProcessorFactorySupplier.get() + .getInstance(component); + processor.setConfiguration(configuration); component.setControlChannelProcessor(processor); } } catch (Exception e) { throw new IllegalStateException("Could not create Dynamic Router endpoint", e); } } else { - final DynamicRouterProcessor processor = processorFactorySupplier.get() - .getInstance("dynamicRouterProcessor-" + configuration.getChannel(), getCamelContext(), - configuration.getRecipientMode(), configuration.isWarnDroppedMessage(), - filterProcessorFactorySupplier); - ServiceHelper.startService(processor); + CamelContext camelContext = getCamelContext(); + String routeId = configuration.getRouteId(); + long timeout = configuration.getTimeout(); + ErrorHandler errorHandler = new NoErrorHandler(null); + if (producerCache == null) { + producerCache = new DefaultProducerCache(this, camelContext, 1000); + } + ExecutorService aggregateExecutorService = camelContext.getExecutorServiceManager() + .newScheduledThreadPool(this, "DynamicRouter-AggregateTask", 0); + if (timeout > 0) { + // use a cached thread pool so we each on-the-fly task has a dedicated thread to process completions as they come in + aggregateExecutorService = camelContext.getExecutorServiceManager() + .newScheduledThreadPool(this, "DynamicRouter-AggregateTask", 0); + } + AggregationStrategy aggregationStrategy = determineAggregationStrategy(camelContext); + DynamicRouterMulticastProcessor processor = processorFactorySupplier.get() + .getInstance("DynamicRouterMulticastProcessor-" + configuration.getChannel(), camelContext, null, + configuration.getRecipientMode(), + configuration.isWarnDroppedMessage(), filterProcessorFactorySupplier, producerCache, + aggregationStrategy, configuration.isParallelProcessing(), + determineExecutorService(camelContext), configuration.isShutdownExecutorService(), + configuration.isStreaming(), configuration.isStopOnException(), timeout, + determineOnPrepare(camelContext), configuration.isShareUnitOfWork(), + configuration.isParallelAggregate()); + processor.setErrorHandler(errorHandler); + processor.setAggregateExecutorService(aggregateExecutorService); + processor.setIgnoreInvalidEndpoints(configuration.isIgnoreInvalidEndpoints()); + processor.setId(getId()); + processor.setRouteId(routeId); + ServiceHelper.startService(aggregationStrategy, producerCache, processor); component.addRoutingProcessor(configuration.getChannel(), processor); } } + protected ExecutorService determineExecutorService(CamelContext camelContext) { + ExecutorService executorService = null; + if (ObjectHelper.isNotEmpty(configuration.getExecutorService())) { + executorService = camelContext.getExecutorServiceManager() + .newThreadPool(this, "@RecipientList", configuration.getExecutorService()); + } + if (configuration.isParallelProcessing() && configuration.getExecutorService() == null) { + // we are running in parallel, so we need a thread pool + executorService = camelContext.getExecutorServiceManager() + .newDefaultThreadPool(this, "@RecipientList"); + } + return executorService; + } + + protected AggregationStrategy determineAggregationStrategy(CamelContext camelContext) { + AggregationStrategy aggregationStrategy = new UseLatestAggregationStrategy(); + if (ObjectHelper.isNotEmpty(configuration.getAggregationStrategy())) { + aggregationStrategy = CamelContextHelper.mandatoryLookup(camelContext, + configuration.getAggregationStrategy(), AggregationStrategy.class); + } + return aggregationStrategy; Review Comment: This is creating an `UseLatestAggregationStrategy` object unnecessarily, which can add overhead to the GC. Try something like this: ``` if (ObjectHelper.isNotEmpty(configuration.getAggregationStrategy())) { return CamelContextHelper.mandatoryLookup(camelContext, configuration.getAggregationStrategy(), AggregationStrategy.class); } // only create a new one if necessary return new UseLatestAggregationStrategy() ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@camel.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org