From d8a7eff866e3f7cc36c308b968f58a0f36fa1413 Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Fri, 16 Jun 2023 14:56:10 +0200 Subject: [PATCH 01/10] First pass - trivial synchronized blocks - Convert the "trivial" synchronized block into reetrant locks. Resolves #8643 --- .../AbstractAmqpOutboundEndpoint.java | 76 ++++-- .../channel/AbstractMessageChannel.java | 11 +- .../channel/DefaultHeaderChannelRegistry.java | 99 ++++--- ...stractSimpleMessageHandlerFactoryBean.java | 12 +- .../config/ConsumerEndpointFactoryBean.java | 21 +- .../config/IdGeneratorConfigurer.java | 38 ++- .../IdempotentReceiverAutoProxyCreator.java | 16 +- ...ourcePollingChannelAdapterFactoryBean.java | 14 +- .../xml/SpelPropertyAccessorsParser.java | 36 ++- .../integration/core/MessagingTemplate.java | 18 +- .../dispatcher/AbstractDispatcher.java | 53 ++-- .../dispatcher/PartitionedDispatcher.java | 11 +- .../endpoint/AbstractPollingEndpoint.java | 13 +- .../endpoint/MethodInvokingMessageSource.java | 13 +- .../endpoint/SourcePollingChannelAdapter.java | 14 +- .../gateway/GatewayMessageHandler.java | 14 +- .../gateway/GatewayProxyFactoryBean.java | 14 +- .../gateway/MessagingGatewaySupport.java | 11 +- .../graph/IntegrationGraphServer.java | 73 +++--- .../AbstractReplyProducingMessageHandler.java | 13 +- .../integration/handler/DelayHandler.java | 41 +-- .../handler/MessageHandlerChain.java | 12 +- .../support/MessagingMethodInvokerHelper.java | 43 +-- .../history/MessageHistoryConfigurer.java | 19 +- .../router/AbstractMessageRouter.java | 11 +- .../selector/MessageSelectorChain.java | 13 +- .../selector/MetadataStoreSelector.java | 13 +- .../store/AbstractMessageGroupStore.java | 39 ++- .../store/PersistentMessageGroup.java | 27 +- .../integration/store/SimpleMessageGroup.java | 13 +- .../support/SmartLifecycleRoleController.java | 19 +- .../channel/BeanFactoryChannelResolver.java | 19 +- .../leader/LockRegistryLeaderInitiator.java | 46 ++-- ...outingSlipHeaderValueMessageProcessor.java | 14 +- .../util/AcceptOnceCollectionFilter.java | 27 +- .../integration/util/SimplePool.java | 147 +++++++---- .../core/TimeBasedUUIDGenerator.java | 14 +- .../feed/inbound/FeedEntryMessageSource.java | 19 +- .../file/FileWritingMessageHandler.java | 99 ++++--- .../config/FileListFilterFactoryBean.java | 14 +- .../filters/AcceptOnceFileListFilter.java | 19 +- .../file/filters/CompositeFileListFilter.java | 40 +-- .../remote/session/CachingSessionFactory.java | 99 ++++--- .../hazelcast/leader/LeaderInitiator.java | 39 ++- ...actHttpRequestExecutingMessageHandler.java | 13 +- ...InternetProtocolSendingMessageHandler.java | 36 ++- .../ip/tcp/TcpSendingMessageHandler.java | 19 +- .../connection/AbstractConnectionFactory.java | 19 +- .../AbstractServerConnectionFactory.java | 9 +- .../CachingClientConnectionFactory.java | 20 +- .../ClientModeConnectionManager.java | 13 +- .../FailoverClientConnectionFactory.java | 146 ++++++----- .../TcpConnectionInterceptorSupport.java | 31 ++- .../ip/tcp/connection/TcpNetConnection.java | 13 +- .../TcpNioClientConnectionFactory.java | 9 +- .../tcp/connection/TcpNioSSLConnection.java | 17 +- .../udp/MulticastReceivingChannelAdapter.java | 43 +-- .../udp/MulticastSendingMessageHandler.java | 21 +- .../udp/UnicastReceivingChannelAdapter.java | 47 ++-- .../ip/udp/UnicastSendingMessageHandler.java | 63 +++-- .../tcp/connection/HelloWorldInterceptor.java | 14 +- .../integration/jdbc/StoredProcExecutor.java | 13 +- ...PostgresChannelMessageTableSubscriber.java | 183 +++++++------ .../jdbc/lock/JdbcLockRegistry.java | 24 +- .../integration/jdbc/LockInterceptor.java | 16 +- .../integration/jpa/test/Consumer.java | 30 ++- .../mqtt/core/AbstractMqttClientManager.java | 35 ++- .../mqtt/core/Mqttv3ClientManager.java | 104 +++++--- .../mqtt/core/Mqttv5ClientManager.java | 94 ++++--- .../MqttPahoMessageDrivenChannelAdapter.java | 99 ++++--- ...Mqttv5PahoMessageDrivenChannelAdapter.java | 23 +- .../mqtt/outbound/MqttPahoMessageHandler.java | 106 ++++---- .../redis/util/RedisLockRegistry.java | 21 +- .../session/DefaultSftpSessionFactory.java | 9 +- .../integration/sftp/session/SftpSession.java | 17 +- .../integration/smb/session/SmbShare.java | 31 ++- .../stomp/outbound/StompMessageHandler.java | 13 +- .../ByteStreamReadingMessageSource.java | 13 +- .../CharacterStreamReadingMessageSource.java | 13 +- .../test/mock/MockMessageHandler.java | 13 +- .../websocket/ClientWebSocketContainer.java | 23 +- .../websocket/ServerWebSocketContainer.java | 17 +- .../ws/AbstractWebServiceOutboundGateway.java | 13 +- .../xml/DefaultXmlPayloadConverter.java | 12 +- .../xml/source/DomSourceFactory.java | 13 +- .../xml/source/StringSourceFactory.java | 13 +- .../ResultToDocumentTransformer.java | 13 +- .../ResultToStringTransformer.java | 13 +- .../config/XmppConnectionFactoryBean.java | 20 +- .../integration/zeromq/ZeroMqProxy.java | 108 ++++---- .../config/CuratorFrameworkFactoryBean.java | 20 +- .../config/LeaderInitiatorFactoryBean.java | 17 +- .../zookeeper/leader/LeaderInitiator.java | 17 +- .../zookeeper/lock/ZookeeperLockRegistry.java | 25 +- .../metadata/ZookeeperMetadataStore.java | 246 +++++++++++------- 95 files changed, 2291 insertions(+), 1085 deletions(-) diff --git a/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java b/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java index b55f77c8707..1b0b3645490 100644 --- a/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java +++ b/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java @@ -1,5 +1,5 @@ /* - * Copyright 2016-2022 the original author or authors. + * Copyright 2016-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,6 +23,8 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.amqp.core.MessageDeliveryMode; import org.springframework.amqp.core.ReturnedMessage; @@ -58,6 +60,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 4.3 * @@ -115,6 +118,8 @@ public abstract class AbstractAmqpOutboundEndpoint extends AbstractReplyProducin private volatile ScheduledFuture confirmChecker; + private final Lock lock = new ReentrantLock(); + /** * Set a custom {@link AmqpHeaderMapper} for mapping request and reply headers. * Defaults to {@link DefaultAmqpHeaderMapper#outboundMapper()}. @@ -336,8 +341,14 @@ public void setConfirmTimeout(long confirmTimeout) { this.confirmTimeout = Duration.ofMillis(confirmTimeout); // NOSONAR sync inconsistency } - protected final synchronized void setConnectionFactory(ConnectionFactory connectionFactory) { - this.connectionFactory = connectionFactory; + protected final void setConnectionFactory(ConnectionFactory connectionFactory) { + this.lock.tryLock(); + try { + this.connectionFactory = connectionFactory; + } + finally { + this.lock.unlock(); + } } protected String getExchangeName() { @@ -487,26 +498,33 @@ protected void endpointInit() { } @Override - public synchronized void start() { - if (!this.running) { - if (!this.lazyConnect && this.connectionFactory != null) { - try { - Connection connection = this.connectionFactory.createConnection(); // NOSONAR (close) - if (connection != null) { - connection.close(); + public void start() { + this.lock.tryLock(); + try { + if (!this.running) { + if (!this.lazyConnect && this.connectionFactory != null) { + try { + Connection connection = this.connectionFactory.createConnection(); // NOSONAR (close) + if (connection != null) { + connection.close(); + } + } + catch (RuntimeException ex) { + logger.error(ex, "Failed to eagerly establish the connection."); } } - catch (RuntimeException ex) { - logger.error(ex, "Failed to eagerly establish the connection."); + doStart(); + if (this.confirmTimeout != null && getConfirmNackChannel() != null && getRabbitTemplate() != null) { + this.confirmChecker = getTaskScheduler() + .scheduleAtFixedRate(checkUnconfirmed(), this.confirmTimeout.dividedBy(2L)); } + this.running = true; } - doStart(); - if (this.confirmTimeout != null && getConfirmNackChannel() != null && getRabbitTemplate() != null) { - this.confirmChecker = getTaskScheduler() - .scheduleAtFixedRate(checkUnconfirmed(), this.confirmTimeout.dividedBy(2L)); - } - this.running = true; } + finally { + this.lock.unlock(); + } + } private Runnable checkUnconfirmed() { @@ -526,14 +544,20 @@ private Runnable checkUnconfirmed() { protected abstract RabbitTemplate getRabbitTemplate(); @Override - public synchronized void stop() { - if (this.running) { - doStop(); - } - this.running = false; - if (this.confirmChecker != null) { - this.confirmChecker.cancel(false); - this.confirmChecker = null; + public void stop() { + this.lock.tryLock(); + try { + if (this.running) { + doStop(); + } + this.running = false; + if (this.confirmChecker != null) { + this.confirmChecker.cancel(false); + this.confirmChecker = null; + } + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java b/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java index 3d0b48dca9e..8b3d5dba3a1 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java @@ -26,6 +26,8 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import io.micrometer.observation.ObservationRegistry; @@ -70,6 +72,7 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ @IntegrationManagedResource public abstract class AbstractMessageChannel extends IntegrationObjectSupport @@ -475,6 +478,8 @@ public void destroy() { */ protected static class ChannelInterceptorList { + private final Lock lock = new ReentrantLock(); + protected final List interceptors = new CopyOnWriteArrayList<>(); // NOSONAR private final LogAccessor logger; @@ -486,11 +491,15 @@ public ChannelInterceptorList(LogAccessor logger) { } public boolean set(List interceptors) { - synchronized (this.interceptors) { + this.lock.tryLock(); + try { this.interceptors.clear(); this.size = interceptors.size(); return this.interceptors.addAll(interceptors); } + finally { + this.lock.unlock(); + } } public int getSize() { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java b/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java index 1e09738224b..1089132738e 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java @@ -1,5 +1,5 @@ /* - * Copyright 2013-2022 the original author or authors. + * Copyright 2013-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.integration.context.IntegrationObjectSupport; import org.springframework.integration.support.channel.HeaderChannelRegistry; @@ -44,6 +46,7 @@ * @author Gary Russell * @author Artem Bilan * @author Trung Pham + * @author Christian Tzolov * * @since 3.0 * @@ -69,6 +72,8 @@ public class DefaultHeaderChannelRegistry extends IntegrationObjectSupport private volatile boolean explicitlyStopped; + private final Lock lock = new ReentrantLock(); + /** * Construct a registry with the default delay for channel expiry. */ @@ -120,25 +125,37 @@ protected void onInit() { } @Override - public synchronized void start() { - if (!this.running) { - Assert.notNull(getTaskScheduler(), "a task scheduler is required"); - this.reaperScheduledFuture = - getTaskScheduler() - .schedule(this, Instant.now().plusMillis(this.reaperDelay)); - - this.running = true; + public void start() { + this.lock.tryLock(); + try { + if (!this.running) { + Assert.notNull(getTaskScheduler(), "a task scheduler is required"); + this.reaperScheduledFuture = getTaskScheduler() + .schedule(this, Instant.now().plusMillis(this.reaperDelay)); + + this.running = true; + } + } + finally { + this.lock.unlock(); } } @Override - public synchronized void stop() { - this.running = false; - if (this.reaperScheduledFuture != null) { - this.reaperScheduledFuture.cancel(true); - this.reaperScheduledFuture = null; + public void stop() { + this.lock.tryLock(); + try { + this.running = false; + if (this.reaperScheduledFuture != null) { + this.reaperScheduledFuture.cancel(true); + this.reaperScheduledFuture = null; + } + this.explicitlyStopped = true; + } + finally { + this.lock.unlock(); } - this.explicitlyStopped = true; + } public void stop(Runnable callback) { @@ -200,35 +217,45 @@ public MessageChannel channelNameToChannel(@Nullable String name) { * Cancel the scheduled reap task and run immediately; then reschedule. */ @Override - public synchronized void runReaper() { - if (this.reaperScheduledFuture != null) { - this.reaperScheduledFuture.cancel(true); - this.reaperScheduledFuture = null; - } + public void runReaper() { + this.lock.tryLock(); + try { + if (this.reaperScheduledFuture != null) { + this.reaperScheduledFuture.cancel(true); + this.reaperScheduledFuture = null; + } - run(); + run(); + } + finally { + this.lock.unlock(); + } } @Override - public synchronized void run() { - logger.trace(() -> "Reaper started; channels size=" + this.channels.size()); - Iterator> iterator = this.channels.entrySet().iterator(); - long now = System.currentTimeMillis(); - while (iterator.hasNext()) { - Entry entry = iterator.next(); - if (entry.getValue().expireAt() < now) { - logger.debug(() -> "Expiring " + entry.getKey() + " (" + entry.getValue().channel() + ")"); - iterator.remove(); + public void run() { + this.lock.tryLock(); + try { + logger.trace(() -> "Reaper started; channels size=" + this.channels.size()); + Iterator> iterator = this.channels.entrySet().iterator(); + long now = System.currentTimeMillis(); + while (iterator.hasNext()) { + Entry entry = iterator.next(); + if (entry.getValue().expireAt() < now) { + logger.debug(() -> "Expiring " + entry.getKey() + " (" + entry.getValue().channel() + ")"); + iterator.remove(); + } } - } - this.reaperScheduledFuture = - getTaskScheduler() - .schedule(this, Instant.now().plusMillis(this.reaperDelay)); + this.reaperScheduledFuture = getTaskScheduler() + .schedule(this, Instant.now().plusMillis(this.reaperDelay)); - logger.trace(() -> "Reaper completed; channels size=" + this.channels.size()); + logger.trace(() -> "Reaper completed; channels size=" + this.channels.size()); + } + finally { + this.lock.unlock(); + } } - protected record MessageChannelWrapper(MessageChannel channel, long expireAt) { } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java index 6784a0a9beb..1312742274d 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.config; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.aopalliance.aop.Advice; import org.apache.commons.logging.Log; @@ -58,6 +60,7 @@ * @author Gary Russell * @author Artem Bilan * @author David Liu + * @author Christian Tzolov */ public abstract class AbstractSimpleMessageHandlerFactoryBean implements FactoryBean, ApplicationContextAware, BeanFactoryAware, BeanNameAware, @@ -65,7 +68,7 @@ public abstract class AbstractSimpleMessageHandlerFactoryBean ((Orderable) this.handler).setOrder(theOrder)); this.initialized = true; + } finally { + this.initializationMonitor.unlock(); } initializingBean(); return this.handler; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java index 809e8b70e5c..a0290020399 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.config; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; import org.aopalliance.aop.Advice; @@ -77,6 +79,7 @@ * @author Josh Long * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class ConsumerEndpointFactoryBean implements FactoryBean, BeanFactoryAware, BeanNameAware, BeanClassLoaderAware, @@ -84,9 +87,9 @@ public class ConsumerEndpointFactoryBean private static final LogAccessor LOGGER = new LogAccessor(LogFactory.getLog(ConsumerEndpointFactoryBean.class)); - private final Object initializationMonitor = new Object(); + private final Lock initializationMonitor = new ReentrantLock(); - private final Object handlerMonitor = new Object(); + private final Lock handlerMonitor = new ReentrantLock(); private MessageHandler handler; @@ -127,7 +130,8 @@ public class ConsumerEndpointFactoryBean public void setHandler(Object handler) { Assert.isTrue(handler instanceof MessageHandler || handler instanceof ReactiveMessageHandler, "'handler' must be an instance of 'MessageHandler' or 'ReactiveMessageHandler'"); - synchronized (this.handlerMonitor) { + this.handlerMonitor.tryLock(); + try { Assert.isNull(this.handler, "handler cannot be overridden"); if (handler instanceof ReactiveMessageHandler) { this.handler = new ReactiveMessageHandlerAdapter((ReactiveMessageHandler) handler); @@ -136,6 +140,9 @@ public void setHandler(Object handler) { this.handler = (MessageHandler) handler; } } + finally { + this.handlerMonitor.unlock(); + } } public MessageHandler getHandler() { @@ -303,7 +310,8 @@ public Class getObjectType() { } private void initializeEndpoint() { - synchronized (this.initializationMonitor) { + this.initializationMonitor.tryLock(); + try { if (this.initialized) { return; } @@ -340,6 +348,9 @@ else if (channel instanceof PollableChannel) { this.endpoint.afterPropertiesSet(); this.initialized = true; } + finally { + this.initializationMonitor.unlock(); + } } private MessageChannel resolveInputChannel() { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java b/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java index 15faf354dba..57483c43324 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2020 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,8 @@ import java.lang.reflect.Field; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -39,11 +41,14 @@ * @author Mark Fisher * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0.4 */ public final class IdGeneratorConfigurer implements ApplicationListener { + private final Lock lock = new ReentrantLock(); + private static final Set GENERATOR_CONTEXT_ID = new HashSet<>(); private static volatile IdGenerator theIdGenerator; @@ -51,21 +56,28 @@ public final class IdGeneratorConfigurer implements ApplicationListener 0; - if (contextHasIdGenerator && setIdGenerator(context)) { - IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.add(context.getId()); + public void onApplicationEvent(ApplicationContextEvent event) { + this.lock.tryLock(); + try { + + ApplicationContext context = event.getApplicationContext(); + if (event instanceof ContextRefreshedEvent) { + boolean contextHasIdGenerator = context.getBeanNamesForType(IdGenerator.class).length > 0; + if (contextHasIdGenerator && setIdGenerator(context)) { + IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.add(context.getId()); + } } - } - else if (event instanceof ContextClosedEvent - && IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.contains(context.getId())) { + else if (event instanceof ContextClosedEvent + && IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.contains(context.getId())) { - if (IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.size() == 1) { - unsetIdGenerator(); + if (IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.size() == 1) { + unsetIdGenerator(); + } + IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.remove(context.getId()); } - IdGeneratorConfigurer.GENERATOR_CONTEXT_ID.remove(context.getId()); + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/IdempotentReceiverAutoProxyCreator.java b/spring-integration-core/src/main/java/org/springframework/integration/config/IdempotentReceiverAutoProxyCreator.java index 4c92d03479f..c97435da9e8 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/IdempotentReceiverAutoProxyCreator.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/IdempotentReceiverAutoProxyCreator.java @@ -1,5 +1,5 @@ /* - * Copyright 2014-2019 the original author or authors. + * Copyright 2014-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.aop.Advisor; import org.springframework.aop.TargetSource; @@ -38,6 +40,8 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov + * * @since 4.1 */ @SuppressWarnings("serial") @@ -47,6 +51,8 @@ class IdempotentReceiverAutoProxyCreator extends AbstractAutoProxyCreator { private volatile Map> idempotentEndpoints; // double check locking requires volatile + private final Lock lock = new ReentrantLock(); + public void setIdempotentEndpointsMapping(List> idempotentEndpointsMapping) { Assert.notEmpty(idempotentEndpointsMapping, "'idempotentEndpointsMapping' must not be empty"); this.idempotentEndpointsMapping = idempotentEndpointsMapping; //NOSONAR (inconsistent sync) @@ -85,8 +91,9 @@ protected Object[] getAdvicesAndAdvisorsForBean(Class beanClass, String beanN } private void initIdempotentEndpointsIfNecessary() { - if (this.idempotentEndpoints == null) { //NOSONAR (inconsistent sync) - synchronized (this) { + if (this.idempotentEndpoints == null) { // NOSONAR (inconsistent sync) + this.lock.tryLock(); + try { if (this.idempotentEndpoints == null) { this.idempotentEndpoints = new LinkedHashMap>(); for (Map mapping : this.idempotentEndpointsMapping) { @@ -104,6 +111,9 @@ private void initIdempotentEndpointsIfNecessary() { } } } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java index 899365a3389..53b71da1f1c 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.config; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.springframework.beans.factory.BeanClassLoaderAware; import org.springframework.beans.factory.BeanFactory; import org.springframework.beans.factory.BeanFactoryAware; @@ -41,11 +44,12 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class SourcePollingChannelAdapterFactoryBean implements FactoryBean, BeanFactoryAware, BeanNameAware, BeanClassLoaderAware, InitializingBean, SmartLifecycle, DisposableBean { - private final Object initializationMonitor = new Object(); + private final Lock initializationMonitor = new ReentrantLock(); private MessageSource source; @@ -158,7 +162,8 @@ public Class getObjectType() { } private void initializeAdapter() { - synchronized (this.initializationMonitor) { + this.initializationMonitor.tryLock(); + try { if (this.initialized) { return; } @@ -207,6 +212,9 @@ private void initializeAdapter() { this.adapter = spca; this.initialized = true; } + finally { + this.initializationMonitor.unlock(); + } } /* diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java b/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java index dbe624f5b60..cbe22790231 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java @@ -1,5 +1,5 @@ /* - * Copyright 2013-2019 the original author or authors. + * Copyright 2013-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.config.xml; import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.w3c.dom.Element; import org.w3c.dom.Node; @@ -38,10 +40,14 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov + * * @since 3.0 */ public class SpelPropertyAccessorsParser implements BeanDefinitionParser { + private final Lock lock = new ReentrantLock(); + private final Map propertyAccessors = new ManagedMap(); @Override @@ -86,17 +92,23 @@ else if (delegate.nodeNameEquals(ele, BeanDefinitionParserDelegate.REF_ELEMENT)) return null; } - private synchronized void initializeSpelPropertyAccessorRegistrarIfNecessary(ParserContext parserContext) { - if (!parserContext.getRegistry() - .containsBeanDefinition(IntegrationContextUtils.SPEL_PROPERTY_ACCESSOR_REGISTRAR_BEAN_NAME)) { - - BeanDefinitionBuilder registrarBuilder = BeanDefinitionBuilder - .genericBeanDefinition(SpelPropertyAccessorRegistrar.class) - .setRole(BeanDefinition.ROLE_INFRASTRUCTURE) - .addConstructorArgValue(this.propertyAccessors); - parserContext.getRegistry() - .registerBeanDefinition(IntegrationContextUtils.SPEL_PROPERTY_ACCESSOR_REGISTRAR_BEAN_NAME, - registrarBuilder.getBeanDefinition()); + private void initializeSpelPropertyAccessorRegistrarIfNecessary(ParserContext parserContext) { + this.lock.tryLock(); + try { + if (!parserContext.getRegistry() + .containsBeanDefinition(IntegrationContextUtils.SPEL_PROPERTY_ACCESSOR_REGISTRAR_BEAN_NAME)) { + + BeanDefinitionBuilder registrarBuilder = BeanDefinitionBuilder + .genericBeanDefinition(SpelPropertyAccessorRegistrar.class) + .setRole(BeanDefinition.ROLE_INFRASTRUCTURE) + .addConstructorArgValue(this.propertyAccessors); + parserContext.getRegistry() + .registerBeanDefinition(IntegrationContextUtils.SPEL_PROPERTY_ACCESSOR_REGISTRAR_BEAN_NAME, + registrarBuilder.getBeanDefinition()); + } + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java b/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java index c0180fafb84..1b7f2ac66a9 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.core; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.springframework.beans.BeansException; import org.springframework.beans.factory.BeanFactory; import org.springframework.integration.context.IntegrationContextUtils; @@ -31,12 +34,15 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 1.0 * */ public class MessagingTemplate extends GenericMessagingTemplate { + private final Lock lock = new ReentrantLock(); + private BeanFactory beanFactory; private volatile boolean throwExceptionOnLateReplySet; @@ -84,15 +90,19 @@ public void setDefaultChannel(MessageChannel channel) { @Nullable public Message sendAndReceive(MessageChannel destination, Message requestMessage) { if (!this.throwExceptionOnLateReplySet) { - synchronized (this) { + this.lock.tryLock(); + try { if (!this.throwExceptionOnLateReplySet) { - IntegrationProperties integrationProperties = - IntegrationContextUtils.getIntegrationProperties(this.beanFactory); + IntegrationProperties integrationProperties = IntegrationContextUtils + .getIntegrationProperties(this.beanFactory); super.setThrowExceptionOnLateReply( integrationProperties.isMessagingTemplateThrowExceptionOnLateReply()); this.throwExceptionOnLateReplySet = true; } } + finally { + this.lock.unlock(); + } } return super.sendAndReceive(destination, requestMessage); } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java index a44984d88c5..d184b05c0b7 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2020 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.dispatcher; import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -41,6 +43,7 @@ * @author Gary Russell * @author Diego Belfer * @author Artem Bilan + * @author Christian Tzolov */ public abstract class AbstractDispatcher implements MessageDispatcher { @@ -52,6 +55,8 @@ public abstract class AbstractDispatcher implements MessageDispatcher { private volatile MessageHandler theOneHandler; + private final Lock lock = new ReentrantLock(); + /** * Set the maximum subscribers allowed by this dispatcher. * @param maxSubscribers The maximum number of subscribers allowed. @@ -77,17 +82,23 @@ protected Set getHandlers() { * @return the result of {@link Set#add(Object)} */ @Override - public synchronized boolean addHandler(MessageHandler handler) { - Assert.notNull(handler, "handler must not be null"); - Assert.isTrue(this.handlers.size() < this.maxSubscribers, "Maximum subscribers exceeded"); - boolean added = this.handlers.add(handler); - if (this.handlers.size() == 1) { - this.theOneHandler = handler; + public boolean addHandler(MessageHandler handler) { + this.lock.tryLock(); + try { + Assert.notNull(handler, "handler must not be null"); + Assert.isTrue(this.handlers.size() < this.maxSubscribers, "Maximum subscribers exceeded"); + boolean added = this.handlers.add(handler); + if (this.handlers.size() == 1) { + this.theOneHandler = handler; + } + else { + this.theOneHandler = null; + } + return added; } - else { - this.theOneHandler = null; + finally { + this.lock.unlock(); } - return added; } /** @@ -96,16 +107,22 @@ public synchronized boolean addHandler(MessageHandler handler) { * @return the result of {@link Set#remove(Object)} */ @Override - public synchronized boolean removeHandler(MessageHandler handler) { - Assert.notNull(handler, "handler must not be null"); - boolean removed = this.handlers.remove(handler); - if (this.handlers.size() == 1) { - this.theOneHandler = this.handlers.iterator().next(); + public boolean removeHandler(MessageHandler handler) { + this.lock.tryLock(); + try { + Assert.notNull(handler, "handler must not be null"); + boolean removed = this.handlers.remove(handler); + if (this.handlers.size() == 1) { + this.theOneHandler = this.handlers.iterator().next(); + } + else { + this.theOneHandler = null; + } + return removed; } - else { - this.theOneHandler = null; + finally { + this.lock.unlock(); } - return removed; } protected boolean tryOptimizedDispatch(Message message) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java index 1883f190fa1..44b40a265e4 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java @@ -25,6 +25,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; import org.springframework.integration.util.ErrorHandlingTaskExecutor; @@ -49,6 +51,7 @@ * The rest of the logic is similar to {@link UnicastingDispatcher} behavior. * * @author Artem Bilan + * @author Christian Tzolov * * @since 6.1 */ @@ -73,6 +76,8 @@ public class PartitionedDispatcher extends AbstractDispatcher { private MessageHandlingTaskDecorator messageHandlingTaskDecorator = task -> task; + private final Lock lock = new ReentrantLock(); + /** * Instantiate based on a provided number of partitions and function for partition key against * the message to dispatch. @@ -153,7 +158,8 @@ public boolean dispatch(Message message) { private void populatedPartitions() { if (this.partitions.isEmpty()) { - synchronized (this.partitions) { + this.lock.tryLock(); + try { if (this.partitions.isEmpty()) { Map partitionsToUse = new HashMap<>(); for (int i = 0; i < this.partitionCount; i++) { @@ -162,6 +168,9 @@ private void populatedPartitions() { this.partitions.putAll(partitionsToUse); } } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java index 68cafafc58e..31b6e4eb2d1 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,8 @@ import java.util.concurrent.Callable; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; import org.aopalliance.aop.Advice; @@ -78,6 +80,7 @@ * @author Gary Russell * @author Artem Bilan * @author Andreas Baer + * @author Christian Tzolov */ public abstract class AbstractPollingEndpoint extends AbstractEndpoint implements BeanClassLoaderAware { @@ -88,7 +91,7 @@ public abstract class AbstractPollingEndpoint extends AbstractEndpoint implement private final Collection appliedAdvices = new HashSet<>(); - private final Object initializationMonitor = new Object(); + private final Lock initializationMonitor = new ReentrantLock(); private Executor taskExecutor = new SyncTaskExecutor(); @@ -262,7 +265,8 @@ protected void setReceiveMessageSource(Object source) { @Override protected void onInit() { - synchronized (this.initializationMonitor) { + this.initializationMonitor.tryLock(); + try { if (this.initialized) { return; } @@ -280,6 +284,9 @@ protected void onInit() { } this.initialized = true; } + finally { + this.initializationMonitor.unlock(); + } try { super.onInit(); } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java index e535ee92e63..c88303beb9c 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.endpoint; import java.lang.reflect.Method; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.Lifecycle; import org.springframework.integration.support.management.ManageableLifecycle; @@ -31,6 +33,7 @@ * @author Mark Fisher * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class MethodInvokingMessageSource extends AbstractMessageSource implements ManageableLifecycle { @@ -42,7 +45,7 @@ public class MethodInvokingMessageSource extends AbstractMessageSource i private volatile boolean initialized; - private final Object initializationMonitor = new Object(); + private final Lock initializationMonitor = new ReentrantLock(); public void setObject(Object object) { @@ -67,7 +70,8 @@ public String getComponentType() { @Override protected void onInit() { - synchronized (this.initializationMonitor) { + this.initializationMonitor.tryLock(); + try { if (this.initialized) { return; } @@ -83,6 +87,9 @@ protected void onInit() { ReflectionUtils.makeAccessible(this.method); this.initialized = true; } + finally { + this.initializationMonitor.unlock(); + } } @Override diff --git a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java index 1c4c5090c82..b75d96ef9c3 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.endpoint; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.springframework.aop.framework.Advised; import org.springframework.beans.factory.BeanCreationException; import org.springframework.context.Lifecycle; @@ -43,6 +46,7 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class SourcePollingChannelAdapter extends AbstractPollingEndpoint implements TrackableComponent { @@ -59,6 +63,8 @@ public class SourcePollingChannelAdapter extends AbstractPollingEndpoint private volatile boolean shouldTrack; + private final Lock lock = new ReentrantLock(); + /** * Specify the source to be polled for Messages. * @@ -175,12 +181,16 @@ protected void onInit() { public MessageChannel getOutputChannel() { if (this.outputChannelName != null) { - synchronized (this) { + this.lock.tryLock(); + try { if (this.outputChannelName != null) { this.outputChannel = getChannelResolver().resolveDestination(this.outputChannelName); this.outputChannelName = null; } } + finally { + this.lock.unlock(); + } } return this.outputChannel; } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java index 4293fe14472..d203ed3b010 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2016-2022 the original author or authors. + * Copyright 2016-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.gateway; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.springframework.beans.factory.BeanCreationException; import org.springframework.beans.factory.BeanFactory; import org.springframework.beans.factory.config.ConfigurableListableBeanFactory; @@ -28,6 +31,7 @@ * The {@link AbstractReplyProducingMessageHandler} implementation for mid-flow Gateway. * * @author Artem Bilan + * @author Christian Tzolov * * @since 5.0 */ @@ -39,6 +43,8 @@ public class GatewayMessageHandler extends AbstractReplyProducingMessageHandler private volatile boolean running; + private final Lock lock = new ReentrantLock(); + public GatewayMessageHandler() { this.gatewayProxyFactoryBean = new GatewayProxyFactoryBean<>(); } @@ -78,11 +84,15 @@ public void setReplyTimeout(Long replyTimeout) { @Override protected Object handleRequestMessage(Message requestMessage) { if (this.exchanger == null) { - synchronized (this) { + this.lock.tryLock(); + try { if (this.exchanger == null) { initialize(); } } + finally { + this.lock.unlock(); + } } return this.exchanger.exchange(requestMessage); } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java index bdb953d78dc..c66305ba44c 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java @@ -29,6 +29,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.Future; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -106,12 +108,13 @@ * @author Gary Russell * @author Artem Bilan * @author JingPeng Xie + * @author Christian Tzolov */ public class GatewayProxyFactoryBean extends AbstractEndpoint implements TrackableComponent, FactoryBean, MethodInterceptor, BeanClassLoaderAware, IntegrationManagement { - private final Object initializationMonitor = new Object(); + private final Lock initializationMonitor = new ReentrantLock(); private final Map gatewayMap = new HashMap<>(); @@ -455,7 +458,8 @@ public void registerMetricsCaptor(MetricsCaptor metricsCaptorToRegister) { @Override @SuppressWarnings("unchecked") protected void onInit() { - synchronized (this.initializationMonitor) { + this.initializationMonitor.tryLock(); + try { if (this.initialized) { return; } @@ -466,13 +470,15 @@ protected void onInit() { populateMethodInvocationGateways(); - ProxyFactory gatewayProxyFactory = - new ProxyFactory(this.serviceInterface, this); + ProxyFactory gatewayProxyFactory = new ProxyFactory(this.serviceInterface, this); gatewayProxyFactory.addAdvice(new DefaultMethodInvokingMethodInterceptor()); this.serviceProxy = (T) gatewayProxyFactory.getProxy(this.beanClassLoader); this.evaluationContext = ExpressionUtils.createStandardEvaluationContext(beanFactory); this.initialized = true; } + finally { + this.initializationMonitor.unlock(); + } } private void populateMethodInvocationGateways() { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java b/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java index 5f048953637..759cdb40f08 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java @@ -19,6 +19,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import io.micrometer.observation.ObservationRegistry; import org.reactivestreams.Publisher; @@ -86,6 +88,7 @@ * @author Gary Russell * @author Artem Bilan * @author Trung Pham + * @author Christian Tzolov */ @IntegrationManagedResource public abstract class MessagingGatewaySupport extends AbstractEndpoint @@ -99,7 +102,7 @@ public abstract class MessagingGatewaySupport extends AbstractEndpoint private final HistoryWritingMessagePostProcessor historyWritingPostProcessor = new HistoryWritingMessagePostProcessor(); - private final Object replyMessageCorrelatorMonitor = new Object(); + private final Lock replyMessageCorrelatorMonitor = new ReentrantLock(); private final ManagementOverrides managementOverrides = new ManagementOverrides(); @@ -892,7 +895,8 @@ private RuntimeException wrapExceptionIfNecessary(Throwable t, String descriptio protected void registerReplyMessageCorrelatorIfNecessary() { MessageChannel replyChan = getReplyChannel(); if (replyChan != null && this.replyMessageCorrelator == null) { - synchronized (this.replyMessageCorrelatorMonitor) { + this.replyMessageCorrelatorMonitor.tryLock(); + try { if (this.replyMessageCorrelator != null) { return; } @@ -923,6 +927,9 @@ else if (replyChan instanceof ReactiveStreamsSubscribableChannel) { correlator.afterPropertiesSet(); this.replyMessageCorrelator = correlator; } + finally { + this.replyMessageCorrelatorMonitor.unlock(); + } if (isRunning()) { this.replyMessageCorrelator.start(); } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java b/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java index 1cd1ecd56ef..52f78e13a09 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java @@ -26,6 +26,8 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -61,6 +63,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 4.3 * @@ -70,6 +73,8 @@ public class IntegrationGraphServer implements ApplicationContextAware, Applicat private static final float GRAPH_VERSION = 1.2f; + private final Lock lock = new ReentrantLock(); + private final NodeFactory nodeFactory = new NodeFactory(this::enhance); private MicrometerNodeEnhancer micrometerEnhancer; @@ -127,12 +132,16 @@ public void onApplicationEvent(ContextRefreshedEvent event) { * @see #rebuild() */ public Graph getGraph() { - if (this.graph == null) { //NOSONAR (sync) - synchronized (this) { + if (this.graph == null) { // NOSONAR (sync) + this.lock.tryLock(); + try { if (this.graph == null) { buildGraph(); } } + finally { + this.lock.unlock(); + } } return this.graph; } @@ -169,35 +178,41 @@ private T enhance(T node) { } } - private synchronized Graph buildGraph() { - if (this.micrometerEnhancer == null && MicrometerMetricsCaptorConfiguration.METER_REGISTRY_PRESENT) { - this.micrometerEnhancer = new MicrometerNodeEnhancer(this.applicationContext); - } - String implementationVersion = IntegrationGraphServer.class.getPackage().getImplementationVersion(); - if (implementationVersion == null) { - implementationVersion = "unknown - is Spring Integration running from the distribution jar?"; - } - Map descriptor = new HashMap<>(); - descriptor.put("provider", "spring-integration"); - descriptor.put("providerVersion", implementationVersion); - descriptor.put("providerFormatVersion", GRAPH_VERSION); - String name = this.applicationName; - if (name == null) { - name = this.applicationContext.getEnvironment().getProperty("spring.application.name"); + private Graph buildGraph() { + this.lock.tryLock(); + try { + if (this.micrometerEnhancer == null && MicrometerMetricsCaptorConfiguration.METER_REGISTRY_PRESENT) { + this.micrometerEnhancer = new MicrometerNodeEnhancer(this.applicationContext); + } + String implementationVersion = IntegrationGraphServer.class.getPackage().getImplementationVersion(); + if (implementationVersion == null) { + implementationVersion = "unknown - is Spring Integration running from the distribution jar?"; + } + Map descriptor = new HashMap<>(); + descriptor.put("provider", "spring-integration"); + descriptor.put("providerVersion", implementationVersion); + descriptor.put("providerFormatVersion", GRAPH_VERSION); + String name = this.applicationName; + if (name == null) { + name = this.applicationContext.getEnvironment().getProperty("spring.application.name"); + } + if (name != null) { + descriptor.put("name", name); + } + this.nodeFactory.reset(); + Collection nodes = new ArrayList<>(); + Collection links = new ArrayList<>(); + Map channelNodes = channels(nodes); + pollingAdapters(nodes, links, channelNodes); + gateways(nodes, links, channelNodes); + producers(nodes, links, channelNodes); + consumers(nodes, links, channelNodes); + this.graph = new Graph(descriptor, nodes, links); + return this.graph; } - if (name != null) { - descriptor.put("name", name); + finally { + this.lock.unlock(); } - this.nodeFactory.reset(); - Collection nodes = new ArrayList<>(); - Collection links = new ArrayList<>(); - Map channelNodes = channels(nodes); - pollingAdapters(nodes, links, channelNodes); - gateways(nodes, links, channelNodes); - producers(nodes, links, channelNodes); - consumers(nodes, links, channelNodes); - this.graph = new Graph(descriptor, nodes, links); - return this.graph; } private Map channels(Collection nodes) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java index d6134f07764..6cec16311b7 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,8 @@ import java.util.LinkedList; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.aopalliance.aop.Advice; @@ -41,10 +43,13 @@ * @author Artem Bilan * @author David Liu * @author Trung Pham + * @author Christian Tzolov */ public abstract class AbstractReplyProducingMessageHandler extends AbstractMessageProducingHandler implements BeanClassLoaderAware { + private final Lock lock = new ReentrantLock(); + private final List adviceChain = new LinkedList<>(); private ClassLoader beanClassLoader = ClassUtils.getDefaultClassLoader(); @@ -72,13 +77,17 @@ protected boolean getRequiresReply() { */ public void setAdviceChain(List adviceChain) { Assert.notEmpty(adviceChain, "adviceChain cannot be empty"); - synchronized (this.adviceChain) { + this.lock.tryLock(); + try { this.adviceChain.clear(); this.adviceChain.addAll(adviceChain); if (isInitialized()) { initAdvisedRequestHandlerIfAny(); } } + finally { + this.lock.unlock(); + } } protected boolean hasAdviceChain() { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java index 74a619faa30..34a5a3614f8 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java @@ -98,6 +98,7 @@ * @author Mark Fisher * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 1.0.3 */ @@ -110,6 +111,8 @@ public class DelayHandler extends AbstractReplyProducingMessageHandler implement public static final long DEFAULT_RETRY_DELAY = 1_000; + private final Lock lock = new ReentrantLock(); + private final ConcurrentMap deliveries = new ConcurrentHashMap<>(); private final Lock removeReleasedMessageLock = new ReentrantLock(); @@ -618,22 +621,28 @@ public int getDelayedMessageCount() { * behavior is dictated by the avoidance of invocation thread overload. */ @Override - public synchronized void reschedulePersistedMessages() { - MessageGroup messageGroup = this.messageStore.getMessageGroup(this.messageGroupId); - try (Stream> messageStream = messageGroup.streamMessages()) { - TaskScheduler taskScheduler = getTaskScheduler(); - messageStream.forEach((message) -> // NOSONAR - taskScheduler.schedule(() -> { - // This is fine to keep the reference to the message, - // because the scheduled task is performed immediately. - long delay = determineDelayForMessage(message); - if (delay > 0) { - releaseMessageAfterDelay(message, delay); - } - else { - releaseMessage(message); - } - }, Instant.now())); + public void reschedulePersistedMessages() { + this.lock.tryLock(); + try { + MessageGroup messageGroup = this.messageStore.getMessageGroup(this.messageGroupId); + try (Stream> messageStream = messageGroup.streamMessages()) { + TaskScheduler taskScheduler = getTaskScheduler(); + messageStream.forEach((message) -> // NOSONAR + taskScheduler.schedule(() -> { + // This is fine to keep the reference to the message, + // because the scheduled task is performed immediately. + long delay = determineDelayForMessage(message); + if (delay > 0) { + releaseMessageAfterDelay(message, delay); + } + else { + releaseMessage(message); + } + }, Instant.now())); + } + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java index 431a9f8f870..e610c06c674 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,7 @@ import java.util.HashSet; import java.util.LinkedList; import java.util.List; +import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.Lifecycle; @@ -66,11 +67,12 @@ * @author Gary Russell * @author Artem Bilan * @author Trung Pham + * @author Christian Tzolov */ public class MessageHandlerChain extends AbstractMessageProducingHandler implements CompositeMessageHandler, ManageableLifecycle { - private final Object initializationMonitor = new Object(); + private final Lock initializationMonitor = new ReentrantLock(); private final ReentrantLock lifecycleLock = new ReentrantLock(); @@ -102,13 +104,17 @@ public IntegrationPatternType getIntegrationPatternType() { @Override protected void onInit() { super.onInit(); - synchronized (this.initializationMonitor) { + this.initializationMonitor.tryLock(); + try { if (!this.initialized) { Assert.notEmpty(this.handlers, "handler list must not be empty"); configureChain(); this.initialized = true; } } + finally { + this.initializationMonitor.unlock(); + } } private void configureChain() { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java index f2b2453ce09..1a5db72dce5 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,6 +34,8 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.Function; @@ -119,6 +121,8 @@ * @author Gary Russell * @author Artem Bilan * @author Trung Pham + * @author Christian Tzolov + * * @since 2.0 */ public class MessagingMethodInvokerHelper extends AbstractExpressionEvaluator implements ManageableLifecycle { @@ -160,6 +164,8 @@ public class MessagingMethodInvokerHelper extends AbstractExpressionEvaluator im SPEL_COMPILERS.put(SpelCompilerMode.MIXED, EXPRESSION_PARSER_MIXED); } + private final Lock lock = new ReentrantLock(); + private final Object targetObject; private final JsonObjectMapper jsonObjectMapper; @@ -486,23 +492,28 @@ private Object processInternal(ParametersWrapper parameters) { } } - private synchronized void initialize() { - if (isProvidedMessageHandlerFactoryBean()) { - LOGGER.trace("Overriding default instance of MessageHandlerMethodFactory with the one provided."); - this.messageHandlerMethodFactory = - getBeanFactory() - .getBean( - this.canProcessMessageList - ? IntegrationContextUtils.LIST_MESSAGE_HANDLER_FACTORY_BEAN_NAME - : IntegrationContextUtils.MESSAGE_HANDLER_FACTORY_BEAN_NAME, - MessageHandlerMethodFactory.class); + private void initialize() { + this.lock.tryLock(); + try { + if (isProvidedMessageHandlerFactoryBean()) { + LOGGER.trace("Overriding default instance of MessageHandlerMethodFactory with the one provided."); + this.messageHandlerMethodFactory = getBeanFactory() + .getBean( + this.canProcessMessageList + ? IntegrationContextUtils.LIST_MESSAGE_HANDLER_FACTORY_BEAN_NAME + : IntegrationContextUtils.MESSAGE_HANDLER_FACTORY_BEAN_NAME, + MessageHandlerMethodFactory.class); + } + else { + configureLocalMessageHandlerFactory(); + } + + prepareEvaluationContext(); + this.initialized = true; } - else { - configureLocalMessageHandlerFactory(); + finally { + this.lock.unlock(); } - - prepareEvaluationContext(); - this.initialized = true; } private boolean isProvidedMessageHandlerFactoryBean() { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java b/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java index 721fffe5fdb..938da033abc 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.util.Collection; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -44,6 +46,7 @@ * @author Mark Fisher * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 2.0 */ @@ -54,6 +57,8 @@ public class MessageHistoryConfigurer implements ManageableSmartLifecycle, BeanF private static final Log LOGGER = LogFactory.getLog(MessageHistoryConfigurer.class); + private final Lock lock = new ReentrantLock(); + private final Set currentlyTrackedComponents = ConcurrentHashMap.newKeySet(); private String[] componentNamePatterns = {"*"}; @@ -180,7 +185,8 @@ public int getPhase() { @ManagedOperation @Override public void start() { - synchronized (this.currentlyTrackedComponents) { + this.lock.tryLock(); + try { if (!this.running) { for (TrackableComponent component : getTrackableComponents(this.beanFactory)) { trackComponentIfAny(component); @@ -188,12 +194,16 @@ public void start() { } } } + finally { + this.lock.unlock(); + } } @ManagedOperation @Override public void stop() { - synchronized (this.currentlyTrackedComponents) { + this.lock.tryLock(); + try { if (this.running) { this.currentlyTrackedComponents.forEach(component -> { component.setShouldTrack(false); @@ -207,6 +217,9 @@ public void stop() { this.running = false; } } + finally { + this.lock.unlock(); + } } private static Collection getTrackableComponents(ListableBeanFactory beanFactory) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java b/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java index f412c9ad386..a5fe7ac5d52 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java @@ -18,6 +18,8 @@ import java.util.Collection; import java.util.UUID; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.beans.factory.BeanFactory; import org.springframework.core.convert.ConversionService; @@ -43,11 +45,14 @@ * @author Soby Chacko * @author Stefan Ferstl * @author Artem Bilan + * @author Christian Tzolov */ @ManagedResource @IntegrationManagedResource public abstract class AbstractMessageRouter extends AbstractMessageHandler implements MessageRouter { + private final Lock lock = new ReentrantLock(); + private final MessagingTemplate messagingTemplate = new MessagingTemplate(); private volatile MessageChannel defaultOutputChannel; @@ -83,12 +88,16 @@ public void setDefaultOutputChannel(MessageChannel defaultOutputChannel) { @Override public MessageChannel getDefaultOutputChannel() { if (this.defaultOutputChannelName != null) { - synchronized (this) { + this.lock.tryLock(); + try { if (this.defaultOutputChannelName != null) { this.defaultOutputChannel = getChannelResolver().resolveDestination(this.defaultOutputChannelName); this.defaultOutputChannelName = null; } } + finally { + this.lock.unlock(); + } } return this.defaultOutputChannel; } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java b/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java index f4824fc7f39..326cf468654 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,8 @@ import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.integration.core.MessageSelector; import org.springframework.messaging.Message; @@ -32,9 +34,12 @@ * @author Mark Fisher * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class MessageSelectorChain implements MessageSelector { + private final Lock lock = new ReentrantLock(); + private volatile VotingStrategy votingStrategy = VotingStrategy.ALL; private final List selectors = new CopyOnWriteArrayList<>(); @@ -72,10 +77,14 @@ public void add(int index, MessageSelector selector) { */ public void setSelectors(List selectors) { Assert.notEmpty(selectors, "selectors must not be empty"); - synchronized (this.selectors) { + this.lock.tryLock(); + try { this.selectors.clear(); this.selectors.addAll(selectors); } + finally { + this.lock.unlock(); + } } /** diff --git a/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java b/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java index 4ec2ac066e5..bc5b51a4f69 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java @@ -1,5 +1,5 @@ /* - * Copyright 2014-2020 the original author or authors. + * Copyright 2014-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,8 @@ package org.springframework.integration.selector; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import org.springframework.integration.core.MessageSelector; @@ -50,11 +52,14 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 4.1 */ public class MetadataStoreSelector implements MessageSelector { + private final Lock lock = new ReentrantLock(); + private final ConcurrentMetadataStore metadataStore; private final MessageProcessor keyStrategy; @@ -119,7 +124,8 @@ public boolean accept(Message message) { return this.metadataStore.putIfAbsent(key, value) == null; } else { - synchronized (this) { + this.lock.tryLock(); + try { String oldValue = this.metadataStore.get(key); if (oldValue == null) { return this.metadataStore.putIfAbsent(key, value) == null; @@ -129,6 +135,9 @@ public boolean accept(Message message) { } return false; } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java b/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java index bb9c7cc727b..4ef214168f4 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,8 @@ import java.util.Arrays; import java.util.Collection; import java.util.LinkedHashSet; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -33,6 +35,7 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ @@ -42,6 +45,8 @@ public abstract class AbstractMessageGroupStore extends AbstractBatchingMessageG protected final Log logger = LogFactory.getLog(getClass()); // NOSONAR final + private final Lock lock = new ReentrantLock(); + private final Collection expiryCallbacks = new LinkedHashSet<>(); private final MessageGroupFactory persistentMessageGroupFactory = @@ -122,22 +127,28 @@ public void registerMessageGroupExpiryCallback(MessageGroupCallback callback) { @Override @ManagedOperation - public synchronized int expireMessageGroups(long timeout) { - int count = 0; - long threshold = System.currentTimeMillis() - timeout; - for (MessageGroup group : this) { - - long timestamp = group.getTimestamp(); - if (this.isTimeoutOnIdle() && group.getLastModified() > 0) { - timestamp = group.getLastModified(); - } + public int expireMessageGroups(long timeout) { + this.lock.tryLock(); + try { + int count = 0; + long threshold = System.currentTimeMillis() - timeout; + for (MessageGroup group : this) { + + long timestamp = group.getTimestamp(); + if (this.isTimeoutOnIdle() && group.getLastModified() > 0) { + timestamp = group.getLastModified(); + } - if (timestamp <= threshold) { - count++; - expire(copy(group)); + if (timestamp <= threshold) { + count++; + expire(copy(group)); + } } + return count; + } + finally { + this.lock.unlock(); } - return count; } /** diff --git a/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java b/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java index 4e1cd00efb4..498eac4f388 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java @@ -1,5 +1,5 @@ /* - * Copyright 2016-2021 the original author or authors. + * Copyright 2016-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ import java.util.Collections; import java.util.Iterator; import java.util.Spliterator; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; import org.apache.commons.logging.Log; @@ -32,6 +34,7 @@ /** * @author Artem Bilan + * @author Christian Tzolov * * @since 4.3 */ @@ -39,6 +42,8 @@ class PersistentMessageGroup implements MessageGroup { private static final Log LOGGER = LogFactory.getLog(PersistentMessageGroup.class); + private final Lock lock = new ReentrantLock(); + private final MessageGroupStore messageGroupStore; private final Collection> messages = new PersistentCollection(); @@ -76,7 +81,8 @@ public Stream> streamMessages() { @Override public Message getOne() { if (this.oneMessage == null) { - synchronized (this) { + this.lock.tryLock(); + try { if (this.oneMessage == null) { if (LOGGER.isDebugEnabled()) { LOGGER.debug("Lazy loading of one message for messageGroup: " + this.original.getGroupId()); @@ -84,6 +90,9 @@ public Message getOne() { this.oneMessage = this.messageGroupStore.getOneMessageFromGroup(this.original.getGroupId()); } } + finally { + this.lock.unlock(); + } } return this.oneMessage; } @@ -109,7 +118,8 @@ public int getSequenceSize() { @Override public int size() { if (this.size == 0) { - synchronized (this) { + this.lock.tryLock(); + try { if (this.size == 0) { if (LOGGER.isDebugEnabled()) { LOGGER.debug("Lazy loading of group size for messageGroup: " + this.original.getGroupId()); @@ -117,6 +127,9 @@ public int size() { this.size = this.messageGroupStore.messageGroupSize(this.original.getGroupId()); } } + finally { + this.lock.unlock(); + } } return this.size; } @@ -195,6 +208,8 @@ public void clear() { private final class PersistentCollection extends AbstractCollection> { + private final Lock innerLock = new ReentrantLock(); + private volatile Collection> collection; PersistentCollection() { @@ -202,7 +217,8 @@ private final class PersistentCollection extends AbstractCollection> private void load() { if (this.collection == null) { - synchronized (this) { + this.innerLock.tryLock(); + try { if (this.collection == null) { Object groupId = PersistentMessageGroup.this.original.getGroupId(); if (LOGGER.isDebugEnabled()) { @@ -211,6 +227,9 @@ private void load() { this.collection = PersistentMessageGroup.this.messageGroupStore.getMessagesForGroup(groupId); } } + finally { + this.innerLock.unlock(); + } } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java b/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java index dc48b6e7252..cd5edeeb15b 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,8 @@ import java.util.Iterator; import java.util.LinkedHashSet; import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.integration.IntegrationMessageHeaderAccessor; import org.springframework.lang.Nullable; @@ -38,11 +40,14 @@ * @author Dave Syer * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ public class SimpleMessageGroup implements MessageGroup { + private final Lock lock = new ReentrantLock(); + private final Object groupId; private final Collection> messages; @@ -189,10 +194,14 @@ public String getCondition() { @Override public Message getOne() { - synchronized (this.messages) { + this.lock.tryLock(); + try { Iterator> iterator = this.messages.iterator(); return iterator.hasNext() ? iterator.next() : null; } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java b/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java index e122610a42c..5bfd3052a69 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2021 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -52,6 +54,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 4.2 * @@ -63,6 +66,8 @@ public class SmartLifecycleRoleController implements ApplicationListener lifecycles = new LinkedMultiValueMap<>(); private final MultiValueMap lazyLifecycles = new LinkedMultiValueMap<>(); @@ -283,9 +288,15 @@ public Map getEndpointsRunningStatus(String role) { Lifecycle::isRunning)); } - private synchronized void addLazyLifecycles() { - this.lazyLifecycles.forEach(this::doAddLifecyclesToRole); - this.lazyLifecycles.clear(); + private void addLazyLifecycles() { + this.lock.tryLock(); + try { + this.lazyLifecycles.forEach(this::doAddLifecyclesToRole); + this.lazyLifecycles.clear(); + } + finally { + this.lock.unlock(); + } } private void doAddLifecyclesToRole(String role, List lifecycleBeanNames) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java b/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java index 1817685c692..a1d53e8986e 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.support.channel; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -39,6 +42,7 @@ * @author Mark Fisher * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @see BeanFactory */ @@ -46,6 +50,8 @@ public class BeanFactoryChannelResolver implements DestinationResolver "Stopped LeaderInitiator for " + getContext()); } - this.future = null; - LOGGER.debug(() -> "Stopped LeaderInitiator for " + getContext()); + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java b/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java index b5277da2173..b8d78effa09 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java @@ -1,5 +1,5 @@ /* - * Copyright 2014-2019 the original author or authors. + * Copyright 2014-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.beans.BeansException; import org.springframework.beans.factory.BeanFactory; @@ -41,12 +43,16 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov + * * @since 4.1 */ public class RoutingSlipHeaderValueMessageProcessor extends AbstractHeaderValueMessageProcessor, Integer>> implements BeanFactoryAware { + private final Lock lock = new ReentrantLock(); + private final List routingSlipPath; private volatile Map, Integer> routingSlip; @@ -79,7 +85,8 @@ public Map, Integer> processMessage(Message message) { // use a local variable to avoid the second access to volatile field on the happy path Map, Integer> slip = this.routingSlip; if (slip == null) { - synchronized (this) { + this.lock.tryLock(); + try { slip = this.routingSlip; if (slip == null) { List slipPath = this.routingSlipPath; @@ -118,6 +125,9 @@ public Map, Integer> processMessage(Message message) { this.routingSlip = slip; } } + finally { + this.lock.unlock(); + } } return slip; } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java b/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java index 3506ddf74a8..256d1d93e58 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; /** * An implementation of {@link CollectionFilter} that remembers the elements passed in @@ -28,6 +30,7 @@ * @param the collection element type. * * @author Mark Fisher + * @author Christian Tzolov * * @since 2.1 */ @@ -35,15 +38,23 @@ public class AcceptOnceCollectionFilter implements CollectionFilter { private volatile Collection lastSeenElements = Collections.emptyList(); - public synchronized Collection filter(Collection unfilteredElements) { - List filteredElements = new ArrayList<>(); - for (T element : unfilteredElements) { - if (!this.lastSeenElements.contains(element)) { - filteredElements.add(element); + private final Lock lock = new ReentrantLock(); + + public Collection filter(Collection unfilteredElements) { + this.lock.tryLock(); + try { + List filteredElements = new ArrayList<>(); + for (T element : unfilteredElements) { + if (!this.lastSeenElements.contains(element)) { + filteredElements.add(element); + } } + this.lastSeenElements = unfilteredElements; + return filteredElements; + } + finally { + this.lock.unlock(); } - this.lastSeenElements = unfilteredElements; - return filteredElements; } } diff --git a/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java b/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java index 1154d54e14a..44642276e9c 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,8 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -40,6 +42,7 @@ * @author Gary Russell * @author Sergey Bogatyrev * @author Artem Bilan + * @author Christian Tzolov * * @since 2.2 * @@ -48,6 +51,8 @@ public class SimplePool implements Pool { protected final Log logger = LogFactory.getLog(getClass()); // NOSONAR final + private final Lock lock = new ReentrantLock(); + private final PoolSemaphore permits = new PoolSemaphore(0); private final AtomicInteger poolSize = new AtomicInteger(); @@ -93,40 +98,46 @@ public SimplePool(int poolSize, PoolItemCallback callback) { * items are returned. * @param poolSize The desired target pool size. */ - public synchronized void setPoolSize(int poolSize) { - int delta = poolSize - this.poolSize.get(); - this.targetPoolSize.addAndGet(delta); - if (this.logger.isDebugEnabled()) { - this.logger.debug(String.format("Target pool size changed by %d, now %d", delta, - this.targetPoolSize.get())); - } - if (delta > 0) { - this.poolSize.addAndGet(delta); - this.permits.release(delta); - } - else { - this.permits.reducePermits(-delta); - - int inUseSize = this.inUse.size(); - int newPoolSize = Math.max(poolSize, inUseSize); - this.poolSize.set(newPoolSize); - - for (int i = this.available.size(); i > newPoolSize - inUseSize; i--) { - T item = this.available.poll(); - if (item != null) { - doRemoveItem(item); - } - else { - break; - } + public void setPoolSize(int poolSize) { + this.lock.tryLock(); + try { + int delta = poolSize - this.poolSize.get(); + this.targetPoolSize.addAndGet(delta); + if (this.logger.isDebugEnabled()) { + this.logger.debug(String.format("Target pool size changed by %d, now %d", delta, + this.targetPoolSize.get())); } + if (delta > 0) { + this.poolSize.addAndGet(delta); + this.permits.release(delta); + } + else { + this.permits.reducePermits(-delta); + + int inUseSize = this.inUse.size(); + int newPoolSize = Math.max(poolSize, inUseSize); + this.poolSize.set(newPoolSize); + + for (int i = this.available.size(); i > newPoolSize - inUseSize; i--) { + T item = this.available.poll(); + if (item != null) { + doRemoveItem(item); + } + else { + break; + } + } - int inUseDelta = poolSize - inUseSize; - if (inUseDelta < 0 && this.logger.isDebugEnabled()) { - this.logger.debug(String.format("Pool is overcommitted by %d; items will be removed when returned", - -inUseDelta)); + int inUseDelta = poolSize - inUseSize; + if (inUseDelta < 0 && this.logger.isDebugEnabled()) { + this.logger.debug(String.format("Pool is overcommitted by %d; items will be removed when returned", + -inUseDelta)); + } } } + finally { + this.lock.unlock(); + } } /** @@ -135,8 +146,14 @@ public synchronized void setPoolSize(int poolSize) { * to be set. */ @Override - public synchronized int getPoolSize() { - return this.poolSize.get(); + public int getPoolSize() { + this.lock.tryLock(); + try { + return this.poolSize.get(); + } + finally { + this.lock.unlock(); + } } @Override @@ -224,36 +241,48 @@ else if (this.callback.isStale(item)) { * Return an item to the pool. */ @Override - public synchronized void releaseItem(T item) { - Assert.notNull(item, "Item cannot be null"); - Assert.isTrue(this.allocated.contains(item), - "You can only release items that were obtained from the pool"); - if (this.inUse.contains(item)) { - if (this.poolSize.get() > this.targetPoolSize.get() || this.closed) { - this.poolSize.decrementAndGet(); - doRemoveItem(item); + public void releaseItem(T item) { + this.lock.tryLock(); + try { + Assert.notNull(item, "Item cannot be null"); + Assert.isTrue(this.allocated.contains(item), + "You can only release items that were obtained from the pool"); + if (this.inUse.contains(item)) { + if (this.poolSize.get() > this.targetPoolSize.get() || this.closed) { + this.poolSize.decrementAndGet(); + doRemoveItem(item); + } + else { + if (this.logger.isDebugEnabled()) { + this.logger.debug("Releasing " + item + " back to the pool"); + } + this.available.add(item); + this.inUse.remove(item); + this.permits.release(); + } } else { if (this.logger.isDebugEnabled()) { - this.logger.debug("Releasing " + item + " back to the pool"); + this.logger.debug("Ignoring release of " + item + " back to the pool - not in use"); } - this.available.add(item); - this.inUse.remove(item); - this.permits.release(); } } - else { - if (this.logger.isDebugEnabled()) { - this.logger.debug("Ignoring release of " + item + " back to the pool - not in use"); - } + finally { + this.lock.unlock(); } } @Override - public synchronized void removeAllIdleItems() { - T item; - while ((item = this.available.poll()) != null) { - doRemoveItem(item); + public void removeAllIdleItems() { + this.lock.tryLock(); + try { + T item; + while ((item = this.available.poll()) != null) { + doRemoveItem(item); + } + } + finally { + this.lock.unlock(); } } @@ -267,9 +296,15 @@ private void doRemoveItem(T item) { } @Override - public synchronized void close() { - this.closed = true; - removeAllIdleItems(); + public void close() { + this.lock.tryLock(); + try { + this.closed = true; + removeAllIdleItems(); + } + finally { + this.lock.unlock(); + } } @SuppressWarnings("serial") diff --git a/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java b/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java index f4f41d93433..cff04681441 100644 --- a/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java +++ b/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,17 +19,20 @@ import java.net.InetAddress; import java.net.NetworkInterface; import java.util.UUID; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.logging.Logger; /** * @author Oleg Zhurakousky * @author Artem Bilan + * @author Christian Tzolov * */ class TimeBasedUUIDGenerator { - static final Object lock = new Object(); + static final Lock lock = new ReentrantLock(); private static final Logger logger = Logger.getLogger(TimeBasedUUIDGenerator.class.getName()); @@ -56,7 +59,8 @@ static UUID generateId() { static UUID generateIdFromTimestamp(long currentTimeMillis) { long time; - synchronized (lock) { + lock.tryLock(); + try { if (currentTimeMillis > lastTime) { lastTime = currentTimeMillis; clockSequence = 0; @@ -65,7 +69,9 @@ static UUID generateIdFromTimestamp(long currentTimeMillis) { ++clockSequence; } } - + finally { + lock.unlock(); + } time = currentTimeMillis; diff --git a/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java b/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java index bb3c0978d9e..318d289ebaf 100644 --- a/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java +++ b/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java @@ -28,6 +28,8 @@ import java.util.List; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import com.rometools.rome.feed.synd.SyndEntry; import com.rometools.rome.feed.synd.SyndFeed; @@ -56,6 +58,7 @@ * @author Oleg Zhurakousky * @author Artem Bilan * @author Aaron Loes + * @author Christian Tzolov * * @since 2.0 */ @@ -69,13 +72,13 @@ public class FeedEntryMessageSource extends AbstractMessageSource { private final Queue entries = new ConcurrentLinkedQueue<>(); - private final Object monitor = new Object(); + private final Lock monitor = new ReentrantLock(); private final Comparator syndEntryComparator = Comparator.comparing(FeedEntryMessageSource::getLastModifiedDate, Comparator.nullsFirst(Comparator.naturalOrder())); - private final Object feedMonitor = new Object(); + private final Lock feedMonitor = new ReentrantLock(); private SyndFeedInput syndFeedInput = new SyndFeedInput(); @@ -176,7 +179,8 @@ protected SyndEntry doReceive() { Assert.isTrue(this.initialized, "'FeedEntryReaderMessageSource' must be initialized before it can produce Messages."); SyndEntry nextEntry; - synchronized (this.monitor) { + this.monitor.tryLock(); + try { nextEntry = getNextEntry(); if (nextEntry == null) { // read feed and try again @@ -184,6 +188,9 @@ protected SyndEntry doReceive() { nextEntry = getNextEntry(); } } + finally { + this.monitor.unlock(); + } return nextEntry; } @@ -225,7 +232,8 @@ private void populateEntryList() { private SyndFeed getFeed() { try { - synchronized (this.feedMonitor) { + this.feedMonitor.tryLock(); + try { SyndFeed feed = buildSyndFeed(); logger.debug(() -> "Retrieved feed for [" + this + "]"); if (feed == null) { @@ -233,6 +241,9 @@ private SyndFeed getFeed() { } return feed; } + finally { + this.feedMonitor.unlock(); + } } catch (Exception e) { throw new MessagingException("Failed to retrieve feed for '" + this + "'", e); diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java b/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java index a8b7bad8e40..273bcc48593 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ import java.util.Set; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -109,6 +110,7 @@ * @author Tony Falabella * @author Alen Turkovic * @author Trung Pham + * @author Christian Tzolov */ public class FileWritingMessageHandler extends AbstractReplyProducingMessageHandler implements ManageableLifecycle, MessageTriggerAction { @@ -130,6 +132,8 @@ public class FileWritingMessageHandler extends AbstractReplyProducingMessageHand PosixFilePermission.OWNER_READ }; + private final Lock lock = new ReentrantLock(); + private final Map fileStates = new HashMap<>(); private final Expression destinationDirectoryExpression; @@ -459,12 +463,16 @@ public void start() { @Override public void stop() { - synchronized (this) { + this.lock.tryLock(); + try { if (this.flushTask != null) { this.flushTask.cancel(true); this.flushTask = null; } } + finally { + this.lock.unlock(); + } Flusher flusher = new Flusher(); flusher.run(); boolean needInterrupt = this.fileStates.size() > 0; @@ -873,37 +881,42 @@ private File evaluateDestinationDirectoryExpression(Message message) { return destinationDirectory; } - private synchronized FileState getFileState(File fileToWriteTo, boolean isString) + private FileState getFileState(File fileToWriteTo, boolean isString) throws FileNotFoundException { - - FileState state; - boolean appendNoFlush = FileExistsMode.APPEND_NO_FLUSH.equals(this.fileExistsMode); - if (appendNoFlush) { - String absolutePath = fileToWriteTo.getAbsolutePath(); - state = this.fileStates.get(absolutePath); - if (state != null // NOSONAR - && ((isString && state.stream != null) || (!isString && state.writer != null))) { - state.close(); - state = null; - this.fileStates.remove(absolutePath); - } - if (state == null) { - if (isString) { - state = new FileState(createWriter(fileToWriteTo, true), - this.lockRegistry.obtain(fileToWriteTo.getAbsolutePath())); + this.lock.tryLock(); + try { + FileState state; + boolean appendNoFlush = FileExistsMode.APPEND_NO_FLUSH.equals(this.fileExistsMode); + if (appendNoFlush) { + String absolutePath = fileToWriteTo.getAbsolutePath(); + state = this.fileStates.get(absolutePath); + if (state != null // NOSONAR + && ((isString && state.stream != null) || (!isString && state.writer != null))) { + state.close(); + state = null; + this.fileStates.remove(absolutePath); } - else { - state = new FileState(createOutputStream(fileToWriteTo, true), - this.lockRegistry.obtain(fileToWriteTo.getAbsolutePath())); + if (state == null) { + if (isString) { + state = new FileState(createWriter(fileToWriteTo, true), + this.lockRegistry.obtain(fileToWriteTo.getAbsolutePath())); + } + else { + state = new FileState(createOutputStream(fileToWriteTo, true), + this.lockRegistry.obtain(fileToWriteTo.getAbsolutePath())); + } + this.fileStates.put(absolutePath, state); } - this.fileStates.put(absolutePath, state); + state.lastWrite = Long.MAX_VALUE; // prevent flush while we write } - state.lastWrite = Long.MAX_VALUE; // prevent flush while we write + else { + state = null; + } + return state; } - else { - state = null; + finally { + this.lock.unlock(); } - return state; } /** @@ -975,7 +988,8 @@ public void flushIfNeeded(MessageFlushPredicate flushPredicate, Message filte private Map findFilesToFlush(MessageFlushPredicate flushPredicate, Message filterMessage) { Map toRemove = new HashMap<>(); - synchronized (this) { + this.lock.tryLock(); + try { Iterator> iterator = this.fileStates.entrySet().iterator(); while (iterator.hasNext()) { Entry entry = iterator.next(); @@ -986,12 +1000,21 @@ private Map findFilesToFlush(MessageFlushPredicate flushPredi } } } + finally { + this.lock.unlock(); + } return toRemove; } - private synchronized void clearState(final File fileToWriteTo, final FileState state) { + private void clearState(final File fileToWriteTo, final FileState state) { if (state != null) { - this.fileStates.remove(fileToWriteTo.getAbsolutePath()); + this.lock.tryLock(); + try { + this.fileStates.remove(fileToWriteTo.getAbsolutePath()); + } + finally { + this.lock.unlock(); + } } } @@ -1014,11 +1037,15 @@ private void doFlush(Map toRemove) { FileWritingMessageHandler.this.logger .debug("Interrupted during flush; not flushed: " + toRestore.keySet()); } - synchronized (this) { + this.lock.tryLock(); + try { for (Entry entry : toRestore.entrySet()) { this.fileStates.putIfAbsent(entry.getKey(), entry.getValue()); } } + finally { + this.lock.unlock(); + } } } @@ -1085,11 +1112,12 @@ private final class Flusher implements Runnable { @Override public void run() { Map toRemove = new HashMap<>(); - synchronized (FileWritingMessageHandler.this) { + FileWritingMessageHandler.this.lock.tryLock(); + try { long expired = FileWritingMessageHandler.this.flushTask == null ? Long.MAX_VALUE : (System.currentTimeMillis() - FileWritingMessageHandler.this.flushInterval); - Iterator> iterator = - FileWritingMessageHandler.this.fileStates.entrySet().iterator(); + Iterator> iterator = FileWritingMessageHandler.this.fileStates.entrySet() + .iterator(); while (iterator.hasNext()) { Entry entry = iterator.next(); FileState state = entry.getValue(); @@ -1100,6 +1128,9 @@ public void run() { } } } + finally { + FileWritingMessageHandler.this.lock.unlock(); + } doFlush(toRemove); } diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java b/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java index a35c19eb0b8..e63bdf53a65 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,8 @@ import java.io.File; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.beans.factory.FactoryBean; import org.springframework.integration.file.filters.AcceptAllFileListFilter; @@ -34,6 +36,8 @@ * @author Mark Fisher * @author Gunnar Hillert * @author Gary Russell + * @author Christian Tzolov + * * @since 1.0.3 */ public class FileListFilterFactoryBean implements FactoryBean> { @@ -52,7 +56,7 @@ public class FileListFilterFactoryBean implements FactoryBean filter) { this.filter = filter; @@ -95,9 +99,13 @@ public void setAlwaysAcceptDirectories(Boolean alwaysAcceptDirectories) { @NonNull public FileListFilter getObject() { if (this.result == null) { - synchronized (this.monitor) { + this.monitor.tryLock(); + try { this.initializeFileListFilter(); } + finally { + this.monitor.unlock(); + } } return this.result; } diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java index f1c4d565167..c8bae3a8a96 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ import java.util.Queue; import java.util.Set; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.lang.Nullable; @@ -37,6 +39,7 @@ * @author Josh Long * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class AcceptOnceFileListFilter extends AbstractFileListFilter implements ReversibleFileListFilter, ResettableFileListFilter { @@ -46,7 +49,7 @@ public class AcceptOnceFileListFilter extends AbstractFileListFilter imple private final Set seenSet = new HashSet(); - private final Object monitor = new Object(); + private final Lock monitor = new ReentrantLock(); /** @@ -69,7 +72,8 @@ public AcceptOnceFileListFilter() { @Override public boolean accept(F file) { - synchronized (this.monitor) { + this.monitor.tryLock(); + try { if (this.seenSet.contains(file)) { return false; } @@ -81,11 +85,15 @@ public boolean accept(F file) { this.seenSet.add(file); return true; } + finally { + this.monitor.unlock(); + } } @Override public void rollback(F file, List files) { - synchronized (this.monitor) { + this.monitor.tryLock(); + try { boolean rollingBack = false; for (F fileToRollback : files) { if (fileToRollback.equals(file)) { @@ -96,6 +104,9 @@ public void rollback(F file, List files) { } } } + finally { + this.monitor.unlock(); + } } @Override diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java index 3eacbcc72a8..7c4fac7a996 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,8 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import org.springframework.beans.factory.InitializingBean; @@ -46,6 +48,7 @@ * @author Josh Long * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class CompositeFileListFilter implements ReversibleFileListFilter, ResettableFileListFilter, DiscardAwareFileListFilter, Closeable { @@ -58,6 +61,7 @@ public class CompositeFileListFilter private boolean oneIsForRecursion; + private final Lock lock = new ReentrantLock(); public CompositeFileListFilter() { this.fileFilters = new LinkedHashSet<>(); @@ -104,24 +108,30 @@ public final CompositeFileListFilter addFilters(FileListFilter... filters) * @param filtersToAdd a list of filters to add * @return this CompositeFileListFilter instance with the added filters */ - public synchronized CompositeFileListFilter addFilters(Collection> filtersToAdd) { - for (FileListFilter elf : filtersToAdd) { - if (elf instanceof DiscardAwareFileListFilter) { - ((DiscardAwareFileListFilter) elf).addDiscardCallback(this.discardCallback); - } - if (elf instanceof InitializingBean) { - try { - ((InitializingBean) elf).afterPropertiesSet(); + public CompositeFileListFilter addFilters(Collection> filtersToAdd) { + this.lock.tryLock(); + try { + for (FileListFilter elf : filtersToAdd) { + if (elf instanceof DiscardAwareFileListFilter) { + ((DiscardAwareFileListFilter) elf).addDiscardCallback(this.discardCallback); } - catch (Exception e) { - throw new IllegalStateException(e); + if (elf instanceof InitializingBean) { + try { + ((InitializingBean) elf).afterPropertiesSet(); + } + catch (Exception e) { + throw new IllegalStateException(e); + } } + this.allSupportAccept = this.allSupportAccept && elf.supportsSingleFileFiltering(); + this.oneIsForRecursion |= elf.isForRecursion(); } - this.allSupportAccept = this.allSupportAccept && elf.supportsSingleFileFiltering(); - this.oneIsForRecursion |= elf.isForRecursion(); + this.fileFilters.addAll(filtersToAdd); + return this; + } + finally { + this.lock.unlock(); } - this.fileFilters.addAll(filtersToAdd); - return this; } @Override diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java b/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java index 2ab2f38416c..44e6604c61c 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,8 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -40,6 +42,7 @@ * @author Gary Russell * @author Alen Turkovic * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ @@ -47,6 +50,8 @@ public class CachingSessionFactory implements SessionFactory, DisposableBe private static final Log LOGGER = LogFactory.getLog(CachingSessionFactory.class); + private final Lock lock = new ReentrantLock(); + private final SessionFactory sessionFactory; private final SimplePool> pool; @@ -146,24 +151,29 @@ public void destroy() { * Clear the cache of sessions; also any in-use sessions will be closed when * returned to the cache. */ - public synchronized void resetCache() { - LOGGER.debug("Cache reset; idle sessions will be removed, in-use sessions will be closed when returned"); - if (this.isSharedSessionCapable && ((SharedSessionCapable) this.sessionFactory).isSharedSession()) { - ((SharedSessionCapable) this.sessionFactory).resetSharedSession(); + public void resetCache() { + this.lock.tryLock(); + try { + LOGGER.debug("Cache reset; idle sessions will be removed, in-use sessions will be closed when returned"); + if (this.isSharedSessionCapable && ((SharedSessionCapable) this.sessionFactory).isSharedSession()) { + ((SharedSessionCapable) this.sessionFactory).resetSharedSession(); + } + long epoch = System.nanoTime(); + /* + * Spin until we get a new value - nano precision but may be lower resolution. We reset the epoch AFTER + * resetting the shared session so there is no possibility of an "old" session being created in the new + * epoch. There is a slight possibility that a "new" session might appear in the old epoch and thus be + * closed when returned to the cache. + */ + while (epoch == this.sharedSessionEpoch) { + epoch = System.nanoTime(); + } + this.sharedSessionEpoch = epoch; + this.pool.removeAllIdleItems(); } - long epoch = System.nanoTime(); - /* - * Spin until we get a new value - nano precision but may be lower resolution. - * We reset the epoch AFTER resetting the shared session so there is no possibility - * of an "old" session being created in the new epoch. There is a slight possibility - * that a "new" session might appear in the old epoch and thus be closed when returned to - * the cache. - */ - while (epoch == this.sharedSessionEpoch) { - epoch = System.nanoTime(); + finally { + this.lock.unlock(); } - this.sharedSessionEpoch = epoch; - this.pool.removeAllIdleItems(); } public class CachedSession implements Session { //NOSONAR must be final, but can't for mocking in tests @@ -174,6 +184,8 @@ public class CachedSession implements Session { //NOSONAR must be final, but private boolean dirty; + private final Lock lock = new ReentrantLock(); + /** * The epoch in which this session was created. */ @@ -185,35 +197,42 @@ private CachedSession(Session targetSession, long sharedSessionEpoch) { } @Override - public synchronized void close() { - if (this.released) { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Session " + this.targetSession + " already released."); - } - } - else { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Releasing Session " + this.targetSession + " back to the pool."); - } - if (this.sharedSessionEpoch != CachingSessionFactory.this.sharedSessionEpoch) { + public void close() { + this.lock.tryLock(); + try { + + if (this.released) { if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Closing session " + this.targetSession + " after reset."); + LOGGER.debug("Session " + this.targetSession + " already released."); } - this.targetSession.close(); } - else if (this.dirty) { - this.targetSession.close(); - } - if (this.targetSession.isOpen()) { - try { - this.targetSession.finalizeRaw(); + else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Releasing Session " + this.targetSession + " back to the pool."); } - catch (IOException e) { - //No-op in this context + if (this.sharedSessionEpoch != CachingSessionFactory.this.sharedSessionEpoch) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Closing session " + this.targetSession + " after reset."); + } + this.targetSession.close(); } + else if (this.dirty) { + this.targetSession.close(); + } + if (this.targetSession.isOpen()) { + try { + this.targetSession.finalizeRaw(); + } + catch (IOException e) { + // No-op in this context + } + } + CachingSessionFactory.this.pool.releaseItem(this.targetSession); + this.released = true; } - CachingSessionFactory.this.pool.releaseItem(this.targetSession); - this.released = true; + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java b/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java index 2a843119cf3..c430c836d67 100644 --- a/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java +++ b/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java @@ -21,6 +21,8 @@ import java.util.concurrent.Future; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import com.hazelcast.core.HazelcastInstance; import com.hazelcast.cp.CPSubsystem; @@ -54,6 +56,7 @@ * @author Mael Le Guével * @author Alexey Tsoy * @author Robert Höglund + * @author Christian Tzolov */ public class LeaderInitiator implements SmartLifecycle, DisposableBean, ApplicationEventPublisherAware { @@ -61,6 +64,8 @@ public class LeaderInitiator implements SmartLifecycle, DisposableBean, Applicat private static final Context NULL_CONTEXT = new NullContext(); + private final Lock lock = new ReentrantLock(); + /*** Hazelcast client. */ private final HazelcastInstance client; @@ -208,11 +213,17 @@ public Context getContext() { * Start the registration of the {@link #candidate} for leader election. */ @Override - public synchronized void start() { - if (!this.running) { - this.leaderSelector = new LeaderSelector(); - this.running = true; - this.future = this.taskExecutor.submit(this.leaderSelector); + public void start() { + this.lock.tryLock(); + try { + if (!this.running) { + this.leaderSelector = new LeaderSelector(); + this.running = true; + this.future = this.taskExecutor.submit(this.leaderSelector); + } + } + finally { + this.lock.unlock(); } } @@ -227,13 +238,19 @@ public void stop(Runnable callback) { * If the candidate is currently leader, its leadership will be revoked. */ @Override - public synchronized void stop() { - if (this.running) { - this.running = false; - if (this.future != null) { - this.future.cancel(true); + public void stop() { + this.lock.tryLock(); + try { + if (this.running) { + this.running = false; + if (this.future != null) { + this.future.cancel(true); + } + this.future = null; } - this.future = null; + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java b/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java index b36ad1e93ba..dd0a5891186 100644 --- a/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java +++ b/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2017-2022 the original author or authors. + * Copyright 2017-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.xml.transform.Source; @@ -71,6 +73,7 @@ * @author Wallace Wadge * @author Shiliang Li * @author Florian Schöffl + * @author Christian Tzolov * * @since 5.0 */ @@ -81,6 +84,8 @@ public abstract class AbstractHttpRequestExecutingMessageHandler extends Abstrac protected final DefaultUriBuilderFactory uriFactory = new DefaultUriBuilderFactory(); // NOSONAR - final + private final Lock lock = new ReentrantLock(); + private final Map uriVariableExpressions = new HashMap<>(); private final Expression uriExpression; @@ -226,10 +231,14 @@ public void setHeaderMapper(HeaderMapper headerMapper) { * @param uriVariableExpressions The URI variable expressions. */ public void setUriVariableExpressions(Map uriVariableExpressions) { - synchronized (this.uriVariableExpressions) { + this.lock.tryLock(); + try { this.uriVariableExpressions.clear(); this.uriVariableExpressions.putAll(uriVariableExpressions); } + finally { + this.lock.unlock(); + } } /** diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java index ef5ff630dd1..1b19c9829d1 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,8 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.integration.handler.AbstractMessageHandler; import org.springframework.integration.support.management.ManageableLifecycle; @@ -27,11 +29,15 @@ * Base class for UDP MessageHandlers. * * @author Gary Russell + * @author Christian Tzolov + * * @since 2.0 */ public abstract class AbstractInternetProtocolSendingMessageHandler extends AbstractMessageHandler implements CommonSocketOptions, ManageableLifecycle { + private final Lock lock = new ReentrantLock(); + private final SocketAddress destinationAddress; private final String host; @@ -119,20 +125,32 @@ public int getSoSendBufferSize() { @Override - public synchronized void start() { - if (!this.running) { - this.doStart(); - this.running = true; + public void start() { + this.lock.tryLock(); + try { + if (!this.running) { + this.doStart(); + this.running = true; + } + } + finally { + this.lock.unlock(); } } protected abstract void doStart(); @Override - public synchronized void stop() { - if (this.running) { - this.doStop(); - this.running = false; + public void stop() { + this.lock.tryLock(); + try { + if (this.running) { + this.doStop(); + this.running = false; + } + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java index 15b3d195218..0ff890aa6ee 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.ApplicationEventPublisher; import org.springframework.integration.handler.AbstractMessageHandler; @@ -48,6 +50,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 * @@ -60,7 +63,7 @@ public class TcpSendingMessageHandler extends AbstractMessageHandler implements */ public static final long DEFAULT_RETRY_INTERVAL = 60000; - protected final Object lifecycleMonitor = new Object(); // NOSONAR + protected final Lock lifecycleMonitor = new ReentrantLock(); // NOSONAR private final Map connections = new ConcurrentHashMap<>(); @@ -251,7 +254,8 @@ protected void onInit() { @Override public void start() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (!this.active) { this.active = true; if (this.clientConnectionFactory != null) { @@ -273,11 +277,15 @@ public void start() { } } } + finally { + this.lifecycleMonitor.unlock(); + } } @Override public void stop() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (this.active) { this.active = false; if (this.scheduledFuture != null) { @@ -292,6 +300,9 @@ public void stop() { } } } + finally { + this.lifecycleMonitor.unlock(); + } } @Override diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java index d7f365e6c7b..a8d7135e67c 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -42,6 +42,8 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.ApplicationEventPublisher; import org.springframework.context.ApplicationEventPublisherAware; @@ -58,6 +60,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 * @@ -73,7 +76,7 @@ public abstract class AbstractConnectionFactory extends IntegrationObjectSupport private static final int DEFAULT_READ_DELAY = 100; - protected final Object lifecycleMonitor = new Object(); // NOSONAR final + protected final Lock lifecycleMonitor = new ReentrantLock(); // NOSONAR final private final Map connections = new ConcurrentHashMap<>(); @@ -546,13 +549,17 @@ protected Executor getTaskExecutor() { if (!this.active) { throw new MessagingException("Connection Factory not started"); } - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (this.taskExecutor == null) { this.privateExecutor = true; this.taskExecutor = Executors.newCachedThreadPool(); } return this.taskExecutor; } + finally { + this.lifecycleMonitor.unlock(); + } } /** @@ -575,7 +582,8 @@ public void stop() { } } } - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (this.privateExecutor) { ExecutorService executorService = (ExecutorService) this.taskExecutor; executorService.shutdown(); @@ -598,6 +606,9 @@ public void stop() { } } } + finally { + this.lifecycleMonitor.unlock(); + } logger.info(() -> "stopped " + this); } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java index 6080df8960b..9abfc8b8f42 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2001-2022 the original author or authors. + * Copyright 2001-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ @@ -75,13 +76,17 @@ public SocketAddress getServerSocketAddress() { @Override public void start() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (!isActive()) { this.setActive(true); this.shuttingDown = false; getTaskExecutor().execute(this); } } + finally { + this.lifecycleMonitor.unlock(); + } super.start(); } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java index b667976f0d7..39422ec0a11 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.util.Map; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.beans.factory.DisposableBean; import org.springframework.core.serializer.Deserializer; @@ -39,11 +41,15 @@ * false, or cache starvation will result. * * @author Gary Russell + * @author Christian Tzolov + * * @since 2.2 * */ public class CachingClientConnectionFactory extends AbstractClientConnectionFactory implements DisposableBean { + private final Lock lock = new ReentrantLock(); + private final AbstractClientConnectionFactory targetConnectionFactory; private final SimplePool pool; @@ -385,9 +391,15 @@ public void start() { } @Override - public synchronized void stop() { - this.targetConnectionFactory.stop(); - this.pool.removeAllIdleItems(); + public void stop() { + this.lock.tryLock(); + try { + this.targetConnectionFactory.stop(); + this.pool.removeAllIdleItems(); + } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java index 663f3223e21..51d31a72664 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.ip.tcp.connection; import java.util.Objects; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -31,6 +33,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.1 * @@ -39,6 +42,8 @@ public class ClientModeConnectionManager implements Runnable { private final Log logger = LogFactory.getLog(this.getClass()); + private final Lock lock = new ReentrantLock(); + private final AbstractConnectionFactory clientConnectionFactory; private volatile TcpConnection lastConnection; @@ -54,7 +59,8 @@ public ClientModeConnectionManager( @Override public void run() { - synchronized (this.clientConnectionFactory) { + this.lock.tryLock(); + try { try { TcpConnection connection = this.clientConnectionFactory.getConnection(); if (!Objects.equals(connection, this.lastConnection)) { @@ -73,6 +79,9 @@ public void run() { this.logger.error("Could not establish connection using " + this.clientConnectionFactory, ex); } } + finally { + this.lock.unlock(); + } } public boolean isConnected() { diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java index 74da615ec37..406dc216853 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ import java.util.List; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.net.ssl.SSLSession; @@ -37,6 +39,8 @@ * succeeds or the list is exhausted. * * @author Gary Russell + * @author Christian Tzolov + * * @since 2.2 * */ @@ -224,6 +228,8 @@ public boolean isRunning() { */ private final class FailoverTcpConnection extends TcpConnectionSupport implements TcpListener { + private final Lock lock = new ReentrantLock(); + private final List connectionFactories; private final String connectionId; @@ -257,46 +263,51 @@ void incrementEpoch() { * factories are down. * @throws InterruptedException if interrupted. */ - private synchronized void findAConnection() throws InterruptedException { - boolean success = false; - AbstractClientConnectionFactory lastFactoryToTry = this.currentFactory; - AbstractClientConnectionFactory nextFactory = null; - if (!this.factoryIterator.hasNext()) { - this.factoryIterator = this.connectionFactories.iterator(); - } - boolean restartedList = false; - while (!success) { - try { - nextFactory = this.factoryIterator.next(); - this.delegate = nextFactory.getConnection(); - if (logger.isDebugEnabled()) { - logger.debug("Got " + this.delegate.getConnectionId() + " from " + nextFactory); - } - this.delegate.registerListener(this); - this.currentFactory = nextFactory; - success = this.delegate.isOpen(); + private void findAConnection() throws InterruptedException { + this.lock.tryLock(); + try { + boolean success = false; + AbstractClientConnectionFactory lastFactoryToTry = this.currentFactory; + AbstractClientConnectionFactory nextFactory = null; + if (!this.factoryIterator.hasNext()) { + this.factoryIterator = this.connectionFactories.iterator(); } - catch (RuntimeException e) { - if (logger.isDebugEnabled()) { - logger.debug(nextFactory + " failed with " - + e.toString() - + ", trying another"); - } - if (restartedList && (lastFactoryToTry == null || lastFactoryToTry.equals(nextFactory))) { - logger.debug("Failover failed to find a connection"); - /* - * We've tried every factory including the - * one the current connection was on. - */ - this.open = false; - throw e; + boolean restartedList = false; + while (!success) { + try { + nextFactory = this.factoryIterator.next(); + this.delegate = nextFactory.getConnection(); + if (logger.isDebugEnabled()) { + logger.debug("Got " + this.delegate.getConnectionId() + " from " + nextFactory); + } + this.delegate.registerListener(this); + this.currentFactory = nextFactory; + success = this.delegate.isOpen(); } - if (!this.factoryIterator.hasNext()) { - this.factoryIterator = this.connectionFactories.iterator(); - restartedList = true; + catch (RuntimeException e) { + if (logger.isDebugEnabled()) { + logger.debug(nextFactory + " failed with " + + e.toString() + + ", trying another"); + } + if (restartedList && (lastFactoryToTry == null || lastFactoryToTry.equals(nextFactory))) { + logger.debug("Failover failed to find a connection"); + /* + * We've tried every factory including the one the current connection was on. + */ + this.open = false; + throw e; + } + if (!this.factoryIterator.hasNext()) { + this.factoryIterator = this.connectionFactories.iterator(); + restartedList = true; + } } } } + finally { + this.lock.unlock(); + } } @Override @@ -316,39 +327,46 @@ public boolean isOpen() { * If send fails on a connection from every factory, we give up. */ @Override - public synchronized void send(Message message) { - boolean success = false; - AbstractClientConnectionFactory lastFactoryToTry = this.currentFactory; - AbstractClientConnectionFactory lastFactoryTried = null; - boolean retried = false; - while (!success) { - try { - lastFactoryTried = this.currentFactory; - this.delegate.send(message); - success = true; - } - catch (RuntimeException e) { - if (retried && lastFactoryTried.equals(lastFactoryToTry)) { - logger.error("All connection factories exhausted", e); - this.open = false; - throw e; - } - retried = true; - if (logger.isDebugEnabled()) { - logger.debug("Send to " + this.delegate.getConnectionId() + " failed; attempting failover", e); - } - this.delegate.close(); + public void send(Message message) { + this.lock.tryLock(); + try { + boolean success = false; + AbstractClientConnectionFactory lastFactoryToTry = this.currentFactory; + AbstractClientConnectionFactory lastFactoryTried = null; + boolean retried = false; + while (!success) { try { - findAConnection(); - } - catch (@SuppressWarnings("unused") InterruptedException e1) { - Thread.currentThread().interrupt(); + lastFactoryTried = this.currentFactory; + this.delegate.send(message); + success = true; } - if (logger.isDebugEnabled()) { - logger.debug("Failing over to " + this.delegate.getConnectionId()); + catch (RuntimeException e) { + if (retried && lastFactoryTried.equals(lastFactoryToTry)) { + logger.error("All connection factories exhausted", e); + this.open = false; + throw e; + } + retried = true; + if (logger.isDebugEnabled()) { + logger.debug("Send to " + this.delegate.getConnectionId() + " failed; attempting failover", + e); + } + this.delegate.close(); + try { + findAConnection(); + } + catch (@SuppressWarnings("unused") InterruptedException e1) { + Thread.currentThread().interrupt(); + } + if (logger.isDebugEnabled()) { + logger.debug("Failing over to " + this.delegate.getConnectionId()); + } } } } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java index 8bde317cc05..03935cf3e87 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java @@ -18,6 +18,8 @@ import java.util.Collections; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.net.ssl.SSLSession; @@ -34,11 +36,14 @@ * * @author Gary Russell * @author Kazuki Shimizu + * @author Christian Tzolov * * @since 2.0 */ public abstract class TcpConnectionInterceptorSupport extends TcpConnectionSupport implements TcpConnectionInterceptor { + private final Lock lock = new ReentrantLock(); + private TcpConnectionSupport theConnection; private TcpListener tcpListener; @@ -238,17 +243,23 @@ public void addNewConnection(TcpConnection connection) { } @Override - public synchronized void removeDeadConnection(TcpConnection connection) { - if (this.removed) { - return; - } - this.removed = true; - if (this.theConnection instanceof TcpConnectionInterceptorSupport && !this.theConnection.equals(this)) { - ((TcpConnectionInterceptorSupport) this.theConnection).removeDeadConnection(this); + public void removeDeadConnection(TcpConnection connection) { + this.lock.tryLock(); + try { + if (this.removed) { + return; + } + this.removed = true; + if (this.theConnection instanceof TcpConnectionInterceptorSupport && !this.theConnection.equals(this)) { + ((TcpConnectionInterceptorSupport) this.theConnection).removeDeadConnection(this); + } + TcpSender sender = getSender(); + if (sender != null && !(sender instanceof TcpConnectionInterceptorSupport)) { + this.interceptedSenders.forEach(snder -> snder.removeDeadConnection(connection)); + } } - TcpSender sender = getSender(); - if (sender != null && !(sender instanceof TcpConnectionInterceptorSupport)) { - this.interceptedSenders.forEach(snder -> snder.removeDeadConnection(connection)); + finally { + this.lock.unlock(); } } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java index b7defc78e8e..c117597c3f2 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java @@ -1,5 +1,5 @@ /* - * Copyright 2001-2021 the original author or authors. + * Copyright 2001-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,8 @@ import java.net.Socket; import java.net.SocketException; import java.net.SocketTimeoutException; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; import javax.net.ssl.SSLSession; @@ -43,12 +45,15 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 * */ public class TcpNetConnection extends TcpConnectionSupport implements SchedulingAwareRunnable { + private final Lock lock = new ReentrantLock(); + private final Socket socket; private volatile OutputStream socketOutputStream; @@ -102,7 +107,8 @@ public boolean isOpen() { @Override @SuppressWarnings("unchecked") - public synchronized void send(Message message) { + public void send(Message message) { + this.lock.tryLock(); try { if (this.socketOutputStream == null) { int writeBufferSize = this.socket.getSendBufferSize(); @@ -121,6 +127,9 @@ public synchronized void send(Message message) { closeConnection(true); throw mex; } + finally { + this.lock.unlock(); + } if (logger.isDebugEnabled()) { logger.debug(getConnectionId() + " Message sent " + message); } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java index 6764db6d349..5fecf63918e 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 * @@ -170,12 +171,16 @@ public void stop() { @Override public void start() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (!isActive()) { setActive(true); getTaskExecutor().execute(this); } } + finally { + this.lifecycleMonitor.unlock(); + } super.start(); } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java index 48c36881d90..95f5e2d040f 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java @@ -21,6 +21,8 @@ import java.nio.channels.SocketChannel; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLEngineResult; @@ -51,6 +53,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.2 * @@ -67,7 +70,7 @@ public class TcpNioSSLConnection extends TcpNioConnection { private final Semaphore semaphore = new Semaphore(0); - private final Object monitorLock = new Object(); + private final Lock monitorLock = new ReentrantLock(); private int handshakeTimeout = DEFAULT_HANDSHAKE_TIMEOUT; @@ -285,12 +288,16 @@ private void initializeEngine() { @Override protected ChannelOutputStream getChannelOutputStream() { - synchronized (this.monitorLock) { + this.monitorLock.tryLock(); + try { if (this.sslChannelOutputStream == null) { this.sslChannelOutputStream = new SSLChannelOutputStream(super.getChannelOutputStream()); } return this.sslChannelOutputStream; } + finally { + this.monitorLock.unlock(); + } } protected SSLChannelOutputStream getSSLChannelOutputStream() { @@ -318,6 +325,8 @@ public void close() { */ final class SSLChannelOutputStream extends ChannelOutputStream { + private final Lock lock = new ReentrantLock(); + private final ChannelOutputStream channelOutputStream; SSLChannelOutputStream(ChannelOutputStream channelOutputStream) { @@ -331,7 +340,8 @@ final class SSLChannelOutputStream extends ChannelOutputStream { * and multiple writes will be necessary. */ @Override - protected synchronized void doWrite(ByteBuffer plainText) throws IOException { + protected void doWrite(ByteBuffer plainText) throws IOException { + this.lock.tryLock(); try { TcpNioSSLConnection.this.writerActive = true; int remaining = plainText.remaining(); @@ -357,6 +367,7 @@ protected synchronized void doWrite(ByteBuffer plainText) throws IOException { } finally { TcpNioSSLConnection.this.writerActive = false; + this.lock.unlock(); } } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java index 74552990099..37bdb9ba544 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2020 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,8 @@ import java.net.InetSocketAddress; import java.net.MulticastSocket; import java.net.NetworkInterface; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.messaging.MessagingException; @@ -32,6 +34,7 @@ * @author Gary Russell * @author Marcin Pilaczynski * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ @@ -39,6 +42,7 @@ public class MulticastReceivingChannelAdapter extends UnicastReceivingChannelAda private final String group; + private final Lock lock = new ReentrantLock(); /** * Constructs a MulticastReceivingChannelAdapter that listens for packets on the @@ -65,24 +69,31 @@ public MulticastReceivingChannelAdapter(String group, int port, boolean lengthCh } @Override - public synchronized DatagramSocket getSocket() { - if (getTheSocket() == null) { - try { - int port = getPort(); - MulticastSocket socket = port == 0 ? new MulticastSocket() : new MulticastSocket(port); - String localAddress = getLocalAddress(); - if (localAddress != null) { - socket.setNetworkInterface(NetworkInterface.getByInetAddress(InetAddress.getByName(localAddress))); + public DatagramSocket getSocket() { + this.lock.tryLock(); + try { + if (getTheSocket() == null) { + try { + int port = getPort(); + MulticastSocket socket = port == 0 ? new MulticastSocket() : new MulticastSocket(port); + String localAddress = getLocalAddress(); + if (localAddress != null) { + socket.setNetworkInterface( + NetworkInterface.getByInetAddress(InetAddress.getByName(localAddress))); + } + setSocketAttributes(socket); + socket.joinGroup(new InetSocketAddress(this.group, 0), null); + setSocket(socket); + } + catch (IOException e) { + throw new MessagingException("failed to create DatagramSocket", e); } - setSocketAttributes(socket); - socket.joinGroup(new InetSocketAddress(this.group, 0), null); - setSocket(socket); - } - catch (IOException e) { - throw new MessagingException("failed to create DatagramSocket", e); } + return super.getSocket(); + } + finally { + this.lock.unlock(); } - return super.getSocket(); } } diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java index e863c3b8821..47af8896029 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2001-2020 the original author or authors. + * Copyright 2001-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,6 +23,8 @@ import java.net.MulticastSocket; import java.net.NetworkInterface; import java.net.URISyntaxException; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.expression.Expression; import org.springframework.messaging.Message; @@ -37,6 +39,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ @@ -48,6 +51,8 @@ public class MulticastSendingMessageHandler extends UnicastSendingMessageHandler private volatile MulticastSocket multicastSocket; + private final Lock lock = new ReentrantLock(); + /** * Constructs a MulticastSendingMessageHandler to send data to the multicast address/port. * @param address The multicast address. @@ -129,11 +134,17 @@ public MulticastSendingMessageHandler(String destinationExpression) { } @Override - protected synchronized DatagramSocket getSocket() throws IOException { - if (getTheSocket() == null) { - createSocket(); + protected DatagramSocket getSocket() throws IOException { + this.lock.tryLock(); + try { + if (getTheSocket() == null) { + createSocket(); + } + return super.getSocket(); + } + finally { + this.lock.unlock(); } - return super.getSocket(); } private void createSocket() throws IOException { diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java index 1f045c7e70b..2497f0aa22b 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,6 +25,8 @@ import java.net.SocketTimeoutException; import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -44,6 +46,7 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ @@ -53,6 +56,8 @@ public class UnicastReceivingChannelAdapter extends AbstractInternetProtocolRece private final DatagramPacketMessageMapper mapper = new DatagramPacketMessageMapper(); + private final Lock lock = new ReentrantLock(); + private DatagramSocket socket; private boolean socketExplicitlySet; @@ -248,27 +253,33 @@ protected DatagramSocket getTheSocket() { return this.socket; } - public synchronized DatagramSocket getSocket() { - if (this.socket == null) { - try { - DatagramSocket datagramSocket; - String localAddress = getLocalAddress(); - int port = super.getPort(); - if (localAddress == null) { - datagramSocket = port == 0 ? new DatagramSocket() : new DatagramSocket(port); + public DatagramSocket getSocket() { + this.lock.tryLock(); + try { + if (this.socket == null) { + try { + DatagramSocket datagramSocket; + String localAddress = getLocalAddress(); + int port = super.getPort(); + if (localAddress == null) { + datagramSocket = port == 0 ? new DatagramSocket() : new DatagramSocket(port); + } + else { + InetAddress whichNic = InetAddress.getByName(localAddress); + datagramSocket = new DatagramSocket(new InetSocketAddress(whichNic, port)); + } + setSocketAttributes(datagramSocket); + this.socket = datagramSocket; } - else { - InetAddress whichNic = InetAddress.getByName(localAddress); - datagramSocket = new DatagramSocket(new InetSocketAddress(whichNic, port)); + catch (IOException e) { + throw new MessagingException("failed to create DatagramSocket", e); } - setSocketAttributes(datagramSocket); - this.socket = datagramSocket; - } - catch (IOException e) { - throw new MessagingException("failed to create DatagramSocket", e); } + return this.socket; + } + finally { + this.lock.unlock(); } - return this.socket; } /** diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java index a5a8c40aa92..44d3425649f 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2001-2022 the original author or authors. + * Copyright 2001-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,6 +34,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.expression.EvaluationContext; import org.springframework.expression.Expression; @@ -58,6 +60,7 @@ * @author Gary Russell * @author Marcin Pilaczynski * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ @@ -66,6 +69,8 @@ public class UnicastSendingMessageHandler extends private static final int DEFAULT_ACK_TIMEOUT = 5000; + private final Lock lock = new ReentrantLock(); + private final DatagramPacketMessageMapper mapper = new DatagramPacketMessageMapper(); private final Map ackControl = Collections.synchronizedMap(new HashMap<>()); @@ -316,7 +321,8 @@ public void handleMessageInternal(Message message) { public void startAckThread() { if (!this.ackThreadRunning) { - synchronized (this) { + this.lock.tryLock(); + try { if (!this.ackThreadRunning) { try { getSocket(); @@ -334,6 +340,9 @@ public void startAckThread() { } } } + finally { + this.lock.unlock(); + } } } @@ -386,28 +395,34 @@ protected DatagramSocket getTheSocket() { return this.socket; } - protected synchronized DatagramSocket getSocket() throws IOException { - if (this.socket == null) { - if (this.acknowledge) { - if (this.localAddress == null) { - this.socket = this.ackPort == 0 ? new DatagramSocket() : new DatagramSocket(this.ackPort); + protected DatagramSocket getSocket() throws IOException { + this.lock.tryLock(); + try { + if (this.socket == null) { + if (this.acknowledge) { + if (this.localAddress == null) { + this.socket = this.ackPort == 0 ? new DatagramSocket() : new DatagramSocket(this.ackPort); + } + else { + InetAddress whichNic = InetAddress.getByName(this.localAddress); + this.socket = new DatagramSocket(new InetSocketAddress(whichNic, this.ackPort)); + } + if (this.soReceiveBufferSize > 0) { + this.socket.setReceiveBufferSize(this.soReceiveBufferSize); + } + logger.debug(() -> "Listening for acks on port: " + getAckPort()); + updateAckAddress(); } else { - InetAddress whichNic = InetAddress.getByName(this.localAddress); - this.socket = new DatagramSocket(new InetSocketAddress(whichNic, this.ackPort)); + this.socket = new DatagramSocket(); } - if (this.soReceiveBufferSize > 0) { - this.socket.setReceiveBufferSize(this.soReceiveBufferSize); - } - logger.debug(() -> "Listening for acks on port: " + getAckPort()); - updateAckAddress(); + setSocketAttributes(this.socket); } - else { - this.socket = new DatagramSocket(); - } - setSocketAttributes(this.socket); + return this.socket; + } + finally { + this.lock.unlock(); } - return this.socket; } protected void updateAckAddress() { @@ -424,8 +439,14 @@ public void setSoReceiveBufferSize(int size) { } @Override - public synchronized void setLocalAddress(String localAddress) { - this.localAddress = localAddress; + public void setLocalAddress(String localAddress) { + this.lock.tryLock(); + try { + this.localAddress = localAddress; + } + finally { + this.lock.unlock(); + } } public void setTaskExecutor(Executor taskExecutor) { diff --git a/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java b/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java index a85d9f890e8..233da519634 100644 --- a/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java +++ b/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,8 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.ApplicationEventPublisher; import org.springframework.integration.support.MessageBuilder; @@ -26,11 +28,15 @@ /** * @author Gary Russell + * @author Christian Tzolov + * * @since 2.0 * */ public class HelloWorldInterceptor extends TcpConnectionInterceptorSupport { + private final Lock lock = new ReentrantLock(); + private volatile boolean negotiated; private final Semaphore negotiationSemaphore = new Semaphore(0); @@ -57,7 +63,8 @@ public HelloWorldInterceptor(String hello, String world, ApplicationEventPublish @Override public boolean onMessage(Message message) { if (!this.negotiated) { - synchronized (this) { + this.lock.tryLock(); + try { if (!this.negotiated) { Object payload = message.getPayload(); logger.debug(this.toString() + " received " + payload); @@ -91,6 +98,9 @@ public boolean onMessage(Message message) { } } } + finally { + this.lock.unlock(); + } } try { return super.onMessage(message); diff --git a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java index 294e0c789d1..f1fa7798f55 100644 --- a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java +++ b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,6 +23,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.sql.DataSource; @@ -51,6 +53,7 @@ * @author Gunnar Hillert * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 2.1 * @@ -63,7 +66,7 @@ public class StoredProcExecutor implements BeanFactoryAware, InitializingBean { private final DataSource dataSource; - private final Object jdbcCallOperationsMapMonitor = new Object(); + private final Lock jdbcCallOperationsMapMonitor = new ReentrantLock(); private Map> returningResultSetRowMappers = new HashMap<>(0); @@ -301,10 +304,14 @@ private Map executeStoredProcedureInternal(Object input, String private SimpleJdbcCallOperations obtainSimpleJdbcCall(String storedProcedureName) { SimpleJdbcCallOperations operations = this.jdbcCallOperationsMap.get(storedProcedureName); if (operations == null) { - synchronized (this.jdbcCallOperationsMapMonitor) { + this.jdbcCallOperationsMapMonitor.tryLock(); + try { operations = this.jdbcCallOperationsMap.computeIfAbsent(storedProcedureName, this::createSimpleJdbcCall); } + finally { + this.jdbcCallOperationsMapMonitor.unlock(); + } } return operations; } diff --git a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java index 08d0c74b4f0..5c9359bd0aa 100644 --- a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java +++ b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java @@ -26,6 +26,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.postgresql.PGNotification; import org.postgresql.jdbc.PgConnection; @@ -61,6 +63,7 @@ * @author Rafael Winterhalter * @author Artem Bilan * @author Igor Lovich + * @author Christian Tzolov * * @since 6.0 */ @@ -68,6 +71,8 @@ public final class PostgresChannelMessageTableSubscriber implements SmartLifecyc private static final LogAccessor LOGGER = new LogAccessor(PostgresChannelMessageTableSubscriber.class); + private final Lock lock = new ReentrantLock(); + private final Map> subscriptionsMap = new ConcurrentHashMap<>(); private final PgConnectionSupplier connectionSupplier; @@ -111,8 +116,14 @@ public PostgresChannelMessageTableSubscriber(PgConnectionSupplier connectionSupp * @deprecated since 6.2 in favor of {@link #setTaskExecutor(AsyncTaskExecutor)} */ @Deprecated(since = "6.2", forRemoval = true) - public synchronized void setExecutor(ExecutorService executor) { - setTaskExecutor(new TaskExecutorAdapter(executor)); + public void setExecutor(ExecutorService executor) { + this.lock.tryLock(); + try { + setTaskExecutor(new TaskExecutorAdapter(executor)); + } + finally { + this.lock.unlock(); + } } /** @@ -149,84 +160,90 @@ public boolean unsubscribe(Subscription subscription) { } @Override - public synchronized void start() { - if (this.latch.getCount() > 0) { - return; - } + public void start() { + this.lock.tryLock(); + try { + if (this.latch.getCount() > 0) { + return; + } - this.latch = new CountDownLatch(1); + this.latch = new CountDownLatch(1); - CountDownLatch startingLatch = new CountDownLatch(1); - this.future = this.taskExecutor.submit(() -> { - try { - while (isActive()) { - try { - PgConnection conn = this.connectionSupplier.get(); - try (Statement stmt = conn.createStatement()) { - stmt.execute("LISTEN " + this.tablePrefix.toLowerCase() + "channel_message_notify"); - } - catch (Exception ex) { - try { - conn.close(); - } - catch (Exception suppressed) { - ex.addSuppressed(suppressed); - } - throw ex; - } - this.subscriptionsMap.values() - .forEach(subscriptions -> subscriptions.forEach(Subscription::notifyUpdate)); + CountDownLatch startingLatch = new CountDownLatch(1); + this.future = this.taskExecutor.submit(() -> { + try { + while (isActive()) { try { - this.connection = conn; - while (isActive()) { - startingLatch.countDown(); - - PGNotification[] notifications = conn.getNotifications(0); - // Unfortunately, there is no good way of interrupting a notification - // poll but by closing its connection. - if (!isActive()) { - return; + PgConnection conn = this.connectionSupplier.get(); + try (Statement stmt = conn.createStatement()) { + stmt.execute("LISTEN " + this.tablePrefix.toLowerCase() + "channel_message_notify"); + } + catch (Exception ex) { + try { + conn.close(); } - if (notifications != null) { - for (PGNotification notification : notifications) { - String parameter = notification.getParameter(); - Set subscriptions = this.subscriptionsMap.get(parameter); - if (subscriptions == null) { - continue; - } - for (Subscription subscription : subscriptions) { - subscription.notifyUpdate(); + catch (Exception suppressed) { + ex.addSuppressed(suppressed); + } + throw ex; + } + this.subscriptionsMap.values() + .forEach(subscriptions -> subscriptions.forEach(Subscription::notifyUpdate)); + try { + this.connection = conn; + while (isActive()) { + startingLatch.countDown(); + + PGNotification[] notifications = conn.getNotifications(0); + // Unfortunately, there is no good way of interrupting a notification + // poll but by closing its connection. + if (!isActive()) { + return; + } + if (notifications != null) { + for (PGNotification notification : notifications) { + String parameter = notification.getParameter(); + Set subscriptions = this.subscriptionsMap.get(parameter); + if (subscriptions == null) { + continue; + } + for (Subscription subscription : subscriptions) { + subscription.notifyUpdate(); + } } } } } + finally { + conn.close(); + } } - finally { - conn.close(); - } - } - catch (Exception e) { - // The getNotifications method does not throw a meaningful message on interruption. - // Therefore, we do not log an error, unless it occurred while active. - if (isActive()) { - LOGGER.error(e, "Failed to poll notifications from Postgres database"); + catch (Exception e) { + // The getNotifications method does not throw a meaningful message on interruption. + // Therefore, we do not log an error, unless it occurred while active. + if (isActive()) { + LOGGER.error(e, "Failed to poll notifications from Postgres database"); + } } } } - } - finally { - this.latch.countDown(); - } - }); + finally { + this.latch.countDown(); + } + }); - try { - if (!startingLatch.await(5, TimeUnit.SECONDS)) { - throw new IllegalStateException("Failed to start " + this); + try { + if (!startingLatch.await(5, TimeUnit.SECONDS)) { + throw new IllegalStateException("Failed to start " + this); + } + } + catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Failed to start " + this, ex); } } - catch (InterruptedException ex) { - Thread.currentThread().interrupt(); - throw new IllegalStateException("Failed to start " + this, ex); + finally { + this.lock.unlock(); } } @@ -239,25 +256,31 @@ private boolean isActive() { } @Override - public synchronized void stop() { - if (this.future.isDone()) { - return; - } - this.future.cancel(true); - PgConnection conn = this.connection; - if (conn != null) { - try { - conn.close(); + public void stop() { + this.lock.tryLock(); + try { + if (this.future.isDone()) { + return; + } + this.future.cancel(true); + PgConnection conn = this.connection; + if (conn != null) { + try { + conn.close(); + } + catch (SQLException ignored) { + } } - catch (SQLException ignored) { + try { + if (!this.latch.await(5, TimeUnit.SECONDS)) { + throw new IllegalStateException("Failed to stop " + this); + } } - } - try { - if (!this.latch.await(5, TimeUnit.SECONDS)) { - throw new IllegalStateException("Failed to stop " + this); + catch (InterruptedException ignored) { } } - catch (InterruptedException ignored) { + finally { + this.lock.unlock(); } } diff --git a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java index f4f01af77a1..85d434dab4f 100644 --- a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java +++ b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java @@ -1,5 +1,5 @@ /* - * Copyright 2016-2022 the original author or authors. + * Copyright 2016-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -54,6 +54,7 @@ * @author Olivier Hubaut * @author Fran Aranda * @author Unseok Kim + * @author Christian Tzolov * * @since 4.3 */ @@ -63,6 +64,8 @@ public class JdbcLockRegistry implements ExpirableLockRegistry, RenewableLockReg private static final int DEFAULT_CAPACITY = 100_000; + private final Lock lock = new ReentrantLock(); + private final Map locks = new LinkedHashMap(16, 0.75F, true) { @@ -111,9 +114,13 @@ public void setCacheCapacity(int cacheCapacity) { public Lock obtain(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = pathFor((String) lockKey); - synchronized (this.locks) { + this.lock.tryLock(); + try { return this.locks.computeIfAbsent(path, key -> new JdbcLock(this.client, this.idleBetweenTries, key)); } + finally { + this.lock.unlock(); + } } private String pathFor(String input) { @@ -123,13 +130,17 @@ private String pathFor(String input) { @Override public void expireUnusedOlderThan(long age) { long now = System.currentTimeMillis(); - synchronized (this.locks) { + this.lock.tryLock(); + try { this.locks.entrySet() .removeIf(entry -> { JdbcLock lock = entry.getValue(); return now - lock.getLastUsed() > age && !lock.isAcquiredInThisProcess(); }); } + finally { + this.lock.unlock(); + } } @Override @@ -137,9 +148,14 @@ public void renewLock(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = pathFor((String) lockKey); JdbcLock jdbcLock; - synchronized (this.locks) { + this.lock.tryLock(); + try { jdbcLock = this.locks.get(path); } + finally { + this.lock.unlock(); + } + if (jdbcLock == null) { throw new IllegalStateException("Could not found mutex at " + path); } diff --git a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java index b5c29573dbc..20cd5957a5c 100644 --- a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java +++ b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ package org.springframework.integration.jdbc; +import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import org.aopalliance.intercept.MethodInterceptor; @@ -23,14 +24,23 @@ /** * @author Dave Syer + * @author Christian Tzolov * */ public class LockInterceptor extends ReentrantLock implements MethodInterceptor { private static final long serialVersionUID = 1L; - public synchronized Object invoke(MethodInvocation invocation) throws Throwable { - return invocation.proceed(); + private final Lock lock = new ReentrantLock(); + + public Object invoke(MethodInvocation invocation) throws Throwable { + this.lock.tryLock(); + try { + return invocation.proceed(); + } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java b/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java index f8e14351106..5d018b7675d 100644 --- a/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java +++ b/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -29,6 +31,8 @@ /** * * @author Gunnar Hillert + * @author Christian Tzolov + * * @since 2.2 * */ @@ -38,13 +42,27 @@ public final class Consumer { private static final BlockingQueue>> MESSAGES = new LinkedBlockingQueue>>(); - public synchronized void receive(Message> message) { - logger.info("Service Activator received Message: " + message); - MESSAGES.add(message); + private final Lock lock = new ReentrantLock(); + + public void receive(Message> message) { + this.lock.tryLock(); + try { + logger.info("Service Activator received Message: " + message); + MESSAGES.add(message); + } + finally { + this.lock.unlock(); + } } - public synchronized Message> poll(long timeoutInMillis) throws InterruptedException { - return MESSAGES.poll(timeoutInMillis, TimeUnit.MILLISECONDS); + public Message> poll(long timeoutInMillis) throws InterruptedException { + this.lock.tryLock(); + try { + return MESSAGES.poll(timeoutInMillis, TimeUnit.MILLISECONDS); + } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java index 2fd2e6b2060..1b176554677 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java @@ -19,6 +19,8 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -38,6 +40,7 @@ * * @author Artem Vozhdayenko * @author Artem Bilan + * @author Christian Tzolov * * @since 6.0 */ @@ -47,6 +50,8 @@ public abstract class AbstractMqttClientManager implements ClientManager connectCallbacks = Collections.synchronizedSet(new HashSet<>()); private final String clientId; @@ -92,8 +97,14 @@ protected ApplicationEventPublisher getApplicationEventPublisher() { return this.applicationEventPublisher; } - protected synchronized void setClient(T client) { - this.client = client; + protected void setClient(T client) { + this.lock.tryLock(); + try { + this.client = client; + } + finally { + this.lock.unlock(); + } } protected Set getCallbacks() { @@ -134,8 +145,14 @@ public boolean isManualAcks() { } @Override - public synchronized T getClient() { - return this.client; + public T getClient() { + this.lock.tryLock(); + try { + return this.client; + } + finally { + this.lock.unlock(); + } } @Override @@ -177,8 +194,14 @@ public boolean removeCallback(ConnectCallback connectCallback) { return this.connectCallbacks.remove(connectCallback); } - public synchronized boolean isRunning() { - return this.client != null; + public boolean isRunning() { + this.lock.tryLock(); + try { + return this.client != null; + } + finally { + this.lock.unlock(); + } } /** diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java index b88416aa803..86ef0328450 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java @@ -16,6 +16,9 @@ package org.springframework.integration.mqtt.core; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.eclipse.paho.client.mqttv3.IMqttAsyncClient; import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttAsyncClient; @@ -36,6 +39,7 @@ * * @author Artem Vozhdayenko * @author Artem Bilan + * @author Christian Tzolov * * @since 6.0 */ @@ -43,6 +47,8 @@ public class Mqttv3ClientManager extends AbstractMqttClientManager implements MqttCallbackExtended { + private final Lock lock = new ReentrantLock(); + private final MqttConnectOptions connectionOptions; private MqttClientPersistence persistence; @@ -87,40 +93,46 @@ public MqttConnectOptions getConnectionInfo() { } @Override - public synchronized void start() { - var client = getClient(); - if (client == null) { - try { - client = createClient(); - } - catch (MqttException e) { - throw new IllegalStateException("could not start client manager", e); - } - } - setClient(client); + public void start() { + this.lock.tryLock(); try { - client.connect(this.connectionOptions).waitForCompletion(getCompletionTimeout()); - } - catch (MqttException ex) { - // See GH-3822 - if (this.connectionOptions.isAutomaticReconnect()) { + var client = getClient(); + if (client == null) { try { - client.reconnect(); + client = createClient(); } - catch (MqttException re) { - logger.error("MQTT client failed to connect. Never happens.", re); + catch (MqttException e) { + throw new IllegalStateException("could not start client manager", e); } } - else { - var applicationEventPublisher = getApplicationEventPublisher(); - if (applicationEventPublisher != null) { - applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, ex)); + setClient(client); + try { + client.connect(this.connectionOptions).waitForCompletion(getCompletionTimeout()); + } + catch (MqttException ex) { + // See GH-3822 + if (this.connectionOptions.isAutomaticReconnect()) { + try { + client.reconnect(); + } + catch (MqttException re) { + logger.error("MQTT client failed to connect. Never happens.", re); + } } else { - logger.error("Could not start client manager, client_id=" + getClientId(), ex); + var applicationEventPublisher = getApplicationEventPublisher(); + if (applicationEventPublisher != null) { + applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, ex)); + } + else { + logger.error("Could not start client manager, client_id=" + getClientId(), ex); + } } } } + finally { + this.lock.unlock(); + } } private IMqttAsyncClient createClient() throws MqttException { @@ -133,31 +145,43 @@ private IMqttAsyncClient createClient() throws MqttException { } @Override - public synchronized void stop() { - var client = getClient(); - if (client == null) { - return; - } + public void stop() { + this.lock.tryLock(); try { - client.disconnectForcibly(getDisconnectCompletionTimeout()); - } - catch (MqttException e) { - logger.error("Could not disconnect from the client", e); - } - finally { + var client = getClient(); + if (client == null) { + return; + } try { - client.close(); + client.disconnectForcibly(getDisconnectCompletionTimeout()); } catch (MqttException e) { - logger.error("Could not close the client", e); + logger.error("Could not disconnect from the client", e); + } + finally { + try { + client.close(); + } + catch (MqttException e) { + logger.error("Could not close the client", e); + } + setClient(null); } - setClient(null); + } + finally { + this.lock.unlock(); } } @Override - public synchronized void connectionLost(Throwable cause) { - logger.error("Connection lost, client_id=" + getClientId(), cause); + public void connectionLost(Throwable cause) { + this.lock.tryLock(); + try { + logger.error("Connection lost, client_id=" + getClientId(), cause); + } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java index 72ec62472d9..167a26d8d1d 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java @@ -16,6 +16,9 @@ package org.springframework.integration.mqtt.core; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.eclipse.paho.mqttv5.client.IMqttAsyncClient; import org.eclipse.paho.mqttv5.client.IMqttToken; import org.eclipse.paho.mqttv5.client.MqttAsyncClient; @@ -38,6 +41,7 @@ * * @author Artem Vozhdayenko * @author Artem Bilan + * @author Christian Tzolov * * @since 6.0 */ @@ -45,6 +49,8 @@ public class Mqttv5ClientManager extends AbstractMqttClientManager implements MqttCallback { + private final Lock lock = new ReentrantLock(); + private final MqttConnectionOptions connectionOptions; private MqttClientPersistence persistence; @@ -89,39 +95,45 @@ public MqttConnectionOptions getConnectionInfo() { } @Override - public synchronized void start() { - var client = getClient(); - if (client == null) { - try { - client = createClient(); - } - catch (MqttException e) { - throw new IllegalStateException("Could not start client manager", e); - } - } - setClient(client); + public void start() { + this.lock.tryLock(); try { - client.connect(this.connectionOptions).waitForCompletion(getCompletionTimeout()); - } - catch (MqttException ex) { - if (this.connectionOptions.isAutomaticReconnect()) { + var client = getClient(); + if (client == null) { try { - client.reconnect(); + client = createClient(); } - catch (MqttException re) { - logger.error("MQTT client failed to connect. Never happens.", re); + catch (MqttException e) { + throw new IllegalStateException("Could not start client manager", e); } } - else { - var applicationEventPublisher = getApplicationEventPublisher(); - if (applicationEventPublisher != null) { - applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, ex)); + setClient(client); + try { + client.connect(this.connectionOptions).waitForCompletion(getCompletionTimeout()); + } + catch (MqttException ex) { + if (this.connectionOptions.isAutomaticReconnect()) { + try { + client.reconnect(); + } + catch (MqttException re) { + logger.error("MQTT client failed to connect. Never happens.", re); + } } else { - logger.error("Could not start client manager, client_id=" + getClientId(), ex); + var applicationEventPublisher = getApplicationEventPublisher(); + if (applicationEventPublisher != null) { + applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, ex)); + } + else { + logger.error("Could not start client manager, client_id=" + getClientId(), ex); + } } } } + finally { + this.lock.unlock(); + } } private MqttAsyncClient createClient() throws MqttException { @@ -134,26 +146,32 @@ private MqttAsyncClient createClient() throws MqttException { } @Override - public synchronized void stop() { - var client = getClient(); - if (client == null) { - return; - } - + public void stop() { + this.lock.tryLock(); try { - client.disconnectForcibly(getDisconnectCompletionTimeout()); - } - catch (MqttException e) { - logger.error("Could not disconnect from the client", e); - } - finally { + var client = getClient(); + if (client == null) { + return; + } + try { - client.close(); + client.disconnectForcibly(getDisconnectCompletionTimeout()); } catch (MqttException e) { - logger.error("Could not close the client", e); + logger.error("Could not disconnect from the client", e); } - setClient(null); + finally { + try { + client.close(); + } + catch (MqttException e) { + logger.error("Could not close the client", e); + } + setClient(null); + } + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java index 26a8f29c89e..71899964556 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java @@ -17,6 +17,8 @@ package org.springframework.integration.mqtt.inbound; import java.util.Arrays; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; import org.eclipse.paho.client.mqttv3.IMqttAsyncClient; @@ -56,6 +58,7 @@ * @author Gary Russell * @author Artem Bilan * @author Artem Vozhdayenko + * @author Christian Tzolov * * @since 4.0 * @@ -64,6 +67,8 @@ public class MqttPahoMessageDrivenChannelAdapter extends AbstractMqttMessageDrivenChannelAdapter implements MqttCallbackExtended, MqttPahoComponent { + private final Lock lock = new ReentrantLock(); + private final MqttPahoClientFactory clientFactory; private volatile IMqttAsyncClient client; @@ -179,46 +184,58 @@ protected void doStart() { } @SuppressWarnings("deprecation") - private synchronized void connect() throws MqttException { - MqttConnectOptions connectionOptions = this.clientFactory.getConnectionOptions(); - var clientManager = getClientManager(); - if (clientManager == null) { - Assert.state(getUrl() != null || connectionOptions.getServerURIs() != null, - "If no 'url' provided, connectionOptions.getServerURIs() must not be null"); - this.client = this.clientFactory.getAsyncClientInstance(getUrl(), getClientId()); - this.client.setCallback(this); - this.client.connect(connectionOptions).waitForCompletion(getCompletionTimeout()); - this.client.setManualAcks(isManualAcks()); + private void connect() throws MqttException { + this.lock.tryLock(); + try { + MqttConnectOptions connectionOptions = this.clientFactory.getConnectionOptions(); + var clientManager = getClientManager(); + if (clientManager == null) { + Assert.state(getUrl() != null || connectionOptions.getServerURIs() != null, + "If no 'url' provided, connectionOptions.getServerURIs() must not be null"); + this.client = this.clientFactory.getAsyncClientInstance(getUrl(), getClientId()); + this.client.setCallback(this); + this.client.connect(connectionOptions).waitForCompletion(getCompletionTimeout()); + this.client.setManualAcks(isManualAcks()); + } + else { + this.client = clientManager.getClient(); + } } - else { - this.client = clientManager.getClient(); + finally { + this.lock.unlock(); } } @Override - protected synchronized void doStop() { - this.readyToSubscribeOnStart = false; + protected void doStop() { + this.lock.tryLock(); try { - if (this.clientFactory.getConnectionOptions().isCleanSession()) { - this.client.unsubscribe(getTopic()); - // Have to re-subscribe on next start if connection is not lost. - this.readyToSubscribeOnStart = true; + this.readyToSubscribeOnStart = false; + try { + if (this.clientFactory.getConnectionOptions().isCleanSession()) { + this.client.unsubscribe(getTopic()); + // Have to re-subscribe on next start if connection is not lost. + this.readyToSubscribeOnStart = true; + } + } + catch (MqttException ex1) { + logger.error(ex1, "Exception while unsubscribing"); } - } - catch (MqttException ex1) { - logger.error(ex1, "Exception while unsubscribing"); - } - if (getClientManager() != null) { - return; - } + if (getClientManager() != null) { + return; + } - try { - this.client.disconnectForcibly(getDisconnectCompletionTimeout()); + try { + this.client.disconnectForcibly(getDisconnectCompletionTimeout()); + } + catch (MqttException ex) { + logger.error(ex, "Exception while disconnecting"); + } } - catch (MqttException ex) { - logger.error(ex, "Exception while disconnecting"); + finally { + this.lock.unlock(); } } @@ -322,17 +339,23 @@ private void warnInvalidQosForSubscription(String[] topics, int[] requestedQos, } @Override - public synchronized void connectionLost(Throwable cause) { - if (isRunning()) { - this.logger.error(() -> "Lost connection: " + cause.getMessage()); - ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); - if (applicationEventPublisher != null) { - applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, cause)); + public void connectionLost(Throwable cause) { + this.lock.tryLock(); + try { + if (isRunning()) { + this.logger.error(() -> "Lost connection: " + cause.getMessage()); + ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); + if (applicationEventPublisher != null) { + applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, cause)); + } + } + else { + // The 'connectComplete()' re-subscribes or sets this flag otherwise. + this.readyToSubscribeOnStart = false; } } - else { - // The 'connectComplete()' re-subscribes or sets this flag otherwise. - this.readyToSubscribeOnStart = false; + finally { + this.lock.unlock(); } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java index cab526ddc5b..65c9eccd2e2 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java @@ -18,6 +18,8 @@ import java.util.Arrays; import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.IntStream; import org.eclipse.paho.mqttv5.client.IMqttAsyncClient; @@ -73,6 +75,7 @@ * @author Lucas Bowler * @author Artem Vozhdayenko * @author Matthias Thoma + * @author Christian Tzolov * * @since 5.5.5 * @@ -81,6 +84,8 @@ public class Mqttv5PahoMessageDrivenChannelAdapter extends AbstractMqttMessageDrivenChannelAdapter implements MqttCallback, MqttComponent { + private final Lock lock = new ReentrantLock(); + private final MqttConnectionOptions connectionOptions; private IMqttAsyncClient mqttClient; @@ -211,13 +216,19 @@ protected void doStart() { } } - private synchronized void connect() throws MqttException { - var clientManager = getClientManager(); - if (clientManager == null) { - this.mqttClient.connect(this.connectionOptions).waitForCompletion(getCompletionTimeout()); + private void connect() throws MqttException { + this.lock.tryLock(); + try { + var clientManager = getClientManager(); + if (clientManager == null) { + this.mqttClient.connect(this.connectionOptions).waitForCompletion(getCompletionTimeout()); + } + else { + this.mqttClient = clientManager.getClient(); + } } - else { - this.mqttClient = clientManager.getClient(); + finally { + this.lock.unlock(); } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java index a9d5f3ab7f2..a720cb83d0c 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.mqtt.outbound; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.eclipse.paho.client.mqttv3.IMqttAsyncClient; import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; @@ -50,6 +53,7 @@ * @author Gary Russell * @author Artem Bilan * @author Artem Vozhdayenko + * @author Christian Tzolov * * @since 4.0 * @@ -57,6 +61,8 @@ public class MqttPahoMessageHandler extends AbstractMqttMessageHandler implements MqttCallback, MqttPahoComponent { + private final Lock lock = new ReentrantLock(); + private final MqttPahoClientFactory clientFactory; private boolean async; @@ -184,41 +190,47 @@ protected void doStop() { } } - private synchronized IMqttAsyncClient checkConnection() throws MqttException { - var theClientManager = getClientManager(); - if (theClientManager != null) { - return theClientManager.getClient(); - } + private IMqttAsyncClient checkConnection() throws MqttException { + this.lock.tryLock(); + try { + var theClientManager = getClientManager(); + if (theClientManager != null) { + return theClientManager.getClient(); + } - if (this.client != null && !this.client.isConnected()) { - this.client.setCallback(null); - this.client.close(); - this.client = null; - } - if (this.client == null) { - try { - MqttConnectOptions connectionOptions = this.clientFactory.getConnectionOptions(); - Assert.state(this.getUrl() != null || connectionOptions.getServerURIs() != null, - "If no 'url' provided, connectionOptions.getServerURIs() must not be null"); - this.client = this.clientFactory.getAsyncClientInstance(this.getUrl(), this.getClientId()); - incrementClientInstance(); - this.client.setCallback(this); - this.client.connect(connectionOptions).waitForCompletion(getCompletionTimeout()); - logger.debug("Client connected"); + if (this.client != null && !this.client.isConnected()) { + this.client.setCallback(null); + this.client.close(); + this.client = null; } - catch (MqttException e) { - if (this.client != null) { - this.client.close(); - this.client = null; + if (this.client == null) { + try { + MqttConnectOptions connectionOptions = this.clientFactory.getConnectionOptions(); + Assert.state(this.getUrl() != null || connectionOptions.getServerURIs() != null, + "If no 'url' provided, connectionOptions.getServerURIs() must not be null"); + this.client = this.clientFactory.getAsyncClientInstance(this.getUrl(), this.getClientId()); + incrementClientInstance(); + this.client.setCallback(this); + this.client.connect(connectionOptions).waitForCompletion(getCompletionTimeout()); + logger.debug("Client connected"); } - ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); - if (applicationEventPublisher != null) { - applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, e)); + catch (MqttException e) { + if (this.client != null) { + this.client.close(); + this.client = null; + } + ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); + if (applicationEventPublisher != null) { + applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, e)); + } + throw new MessagingException("Failed to connect", e); } - throw new MessagingException("Failed to connect", e); } + return this.client; + } + finally { + this.lock.unlock(); } - return this.client; } @Override @@ -252,22 +264,28 @@ private void sendDeliveryComplete(IMqttDeliveryToken token) { } @Override - public synchronized void connectionLost(Throwable cause) { - logger.error("Lost connection; will attempt reconnect on next request"); - if (this.client != null) { - try { - this.client.setCallback(null); - this.client.close(); - } - catch (MqttException e) { - // NOSONAR - } - this.client = null; - ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); - if (applicationEventPublisher != null) { - applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, cause)); + public void connectionLost(Throwable cause) { + this.lock.tryLock(); + try { + logger.error("Lost connection; will attempt reconnect on next request"); + if (this.client != null) { + try { + this.client.setCallback(null); + this.client.close(); + } + catch (MqttException e) { + // NOSONAR + } + this.client = null; + ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); + if (applicationEventPublisher != null) { + applicationEventPublisher.publishEvent(new MqttConnectionFailedEvent(this, cause)); + } } } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java b/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java index 2f6e3ecddc6..d447a41d8e9 100644 --- a/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java +++ b/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java @@ -82,6 +82,7 @@ * @author Vedran Pavic * @author Unseok Kim * @author Anton Gabov + * @author Christian Tzolov * * @since 4.0 * @@ -94,6 +95,8 @@ public final class RedisLockRegistry implements ExpirableLockRegistry, Disposabl private static final int DEFAULT_CAPACITY = 100_000; + private final Lock lock = new ReentrantLock(); + private final Map locks = new LinkedHashMap(16, 0.75F, true) { @@ -224,15 +227,20 @@ public void setRedisLockType(RedisLockType redisLockType) { public Lock obtain(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = (String) lockKey; - synchronized (this.locks) { + this.lock.tryLock(); + try { return this.locks.computeIfAbsent(path, getRedisLockConstructor(this.redisLockType)); } + finally { + this.lock.unlock(); + } } @Override public void expireUnusedOlderThan(long age) { long now = System.currentTimeMillis(); - synchronized (this.locks) { + this.lock.tryLock(); + try { this.locks.entrySet() .removeIf(entry -> { RedisLock lock = entry.getValue(); @@ -243,6 +251,9 @@ public void expireUnusedOlderThan(long age) { && !lock.isAcquiredInThisProcess(); }); } + finally { + this.lock.unlock(); + } } @Override @@ -631,7 +642,8 @@ private boolean subscribeLock(long time) throws ExecutionException, InterruptedE } private void runRedisMessageListenerContainer() { - synchronized (RedisLockRegistry.this.locks) { + RedisLockRegistry.this.lock.tryLock(); + try { if (!(RedisLockRegistry.this.isRunningRedisMessageListenerContainer && RedisLockRegistry.this.redisMessageListenerContainer != null && RedisLockRegistry.this.redisMessageListenerContainer.isRunning())) { @@ -645,6 +657,9 @@ private void runRedisMessageListenerContainer() { RedisLockRegistry.this.isRunningRedisMessageListenerContainer = true; } } + finally { + RedisLockRegistry.this.lock.unlock(); + } } private static final class RedisUnLockNotifyMessageListener implements MessageListener { diff --git a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java index b74c1312ab2..11dbd83794c 100644 --- a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java +++ b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java @@ -62,11 +62,14 @@ * @author Artem Bilan * @author Krzysztof Debski * @author Auke Zaaiman + * @author Christian Tzolov * * @since 2.0 */ public class DefaultSftpSessionFactory implements SessionFactory, SharedSessionCapable { + private final Lock lock = new ReentrantLock(); + private final SshClient sshClient; private volatile boolean initialized; @@ -323,12 +326,16 @@ private ClientSession initClientSession() throws IOException { private void initClient() throws IOException { if (!this.initialized) { - synchronized (this) { + this.lock.tryLock(); + try { if (!this.initialized) { doInitClient(); this.initialized = true; } } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java index 819d257b125..950a6a3ee2e 100644 --- a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java +++ b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java @@ -22,6 +22,8 @@ import java.io.UncheckedIOException; import java.net.SocketAddress; import java.time.Duration; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -46,11 +48,14 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 */ public class SftpSession implements Session { + private final Lock lock = new ReentrantLock(); + private final SftpClient sftpClient; public SftpSession(SftpClient sftpClient) { @@ -122,15 +127,20 @@ public boolean finalizeRaw() { @Override public void write(InputStream inputStream, String destination) throws IOException { - synchronized (this.sftpClient) { + this.lock.tryLock(); + try { OutputStream outputStream = this.sftpClient.write(destination); FileCopyUtils.copy(inputStream, outputStream); } + finally { + this.lock.unlock(); + } } @Override public void append(InputStream inputStream, String destination) throws IOException { - synchronized (this.sftpClient) { + this.lock.tryLock(); + try { OutputStream outputStream = this.sftpClient.write(destination, SftpClient.OpenMode.Create, @@ -138,6 +148,9 @@ public void append(InputStream inputStream, String destination) throws IOExcepti SftpClient.OpenMode.Append); FileCopyUtils.copy(inputStream, outputStream); } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java b/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java index 3878f0223e5..05f12e43def 100644 --- a/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java +++ b/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java @@ -1,5 +1,5 @@ /* - * Copyright 2012-2022 the original author or authors. + * Copyright 2012-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,8 @@ import java.io.IOException; import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import jcifs.CIFSContext; import jcifs.CIFSException; @@ -41,6 +43,7 @@ * @author Gregory Bragg * @author Adam Jones * @author Artem Bilan + * @author Christian Tzolov * * @since 6.0 */ @@ -48,6 +51,8 @@ public class SmbShare extends SmbFile { private static final Log logger = LogFactory.getLog(SmbShare.class); + private final Lock lock = new ReentrantLock(); + private final AtomicBoolean open = new AtomicBoolean(false); private final AtomicBoolean closeContext = new AtomicBoolean(false); @@ -126,17 +131,23 @@ boolean isOpened() { } @Override - public synchronized void close() { - this.open.set(false); - if (this.closeContext.get()) { - try { - getContext().close(); - } - catch (CIFSException e) { - logger.error("Unable to close share: " + this); + public void close() { + this.lock.tryLock(); + try { + this.open.set(false); + if (this.closeContext.get()) { + try { + getContext().close(); + } + catch (CIFSException e) { + logger.error("Unable to close share: " + this); + } } + super.close(); + } + finally { + this.lock.unlock(); } - super.close(); } /** diff --git a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java index 7c93fa2de03..48bd346de70 100644 --- a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java +++ b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2020 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,8 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.ApplicationEventPublisher; import org.springframework.context.ApplicationEventPublisherAware; @@ -51,6 +53,7 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 4.2 */ @@ -59,6 +62,8 @@ public class StompMessageHandler extends AbstractMessageHandler private static final int DEFAULT_CONNECT_TIMEOUT = 3000; + private final Lock lock = new ReentrantLock(); + private final StompSessionHandler sessionHandler = new IntegrationOutboundStompSessionHandler(); private final StompSessionManager stompSessionManager; @@ -178,7 +183,8 @@ protected void handleMessageInternal(final Message message) { } private void connectIfNecessary() throws InterruptedException { - synchronized (this.connectSemaphore) { + this.lock.tryLock(); + try { if (this.stompSession == null || !this.stompSessionManager.isConnected()) { this.stompSessionManager.disconnect(this.sessionHandler); this.stompSessionManager.connect(this.sessionHandler); @@ -199,6 +205,9 @@ private void connectIfNecessary() throws InterruptedException { } } } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java b/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java index b5a15f814eb..6e6ddf17278 100644 --- a/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java +++ b/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,8 @@ import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.integration.endpoint.AbstractMessageSource; import org.springframework.messaging.MessagingException; @@ -28,9 +30,12 @@ * * @author Mark Fisher * @author Artem Bilan + * @author Christian Tzolov */ public class ByteStreamReadingMessageSource extends AbstractMessageSource { + private final Lock lock = new ReentrantLock(); + private final BufferedInputStream stream; private int bytesPerMessage = 1024; // NOSONAR magic number @@ -73,13 +78,17 @@ protected byte[] doReceive() { try { byte[] bytes; int bytesRead = 0; - synchronized (this.stream) { + this.lock.tryLock(); + try { if (this.stream.available() == 0) { return null; } bytes = new byte[this.bytesPerMessage]; bytesRead = this.stream.read(bytes, 0, bytes.length); } + finally { + this.lock.unlock(); + } if (bytesRead <= 0) { return null; } diff --git a/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java b/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java index 450394047ff..43fde4da39a 100644 --- a/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java +++ b/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ import java.io.InputStreamReader; import java.io.Reader; import java.io.UnsupportedEncodingException; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.ApplicationEventPublisher; import org.springframework.context.ApplicationEventPublisherAware; @@ -34,10 +36,13 @@ * @author Mark Fisher * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class CharacterStreamReadingMessageSource extends AbstractMessageSource implements ApplicationEventPublisherAware { + private final Lock lock = new ReentrantLock(); + private final BufferedReader reader; private final boolean blockToDetectEOF; @@ -112,7 +117,8 @@ public String getComponentType() { @Override public String doReceive() { try { - synchronized (this.reader) { + this.lock.tryLock(); + try { if (!this.blockToDetectEOF && !this.reader.ready()) { return null; } @@ -122,6 +128,9 @@ public String doReceive() { } return line; } + finally { + this.lock.unlock(); + } } catch (IOException e) { throw new MessagingException("IO failure occurred in adapter", e); diff --git a/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java b/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java index 646d97c72b6..6310e21377a 100644 --- a/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java +++ b/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java @@ -1,5 +1,5 @@ /* - * Copyright 2017-2019 the original author or authors. + * Copyright 2017-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,8 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.Function; @@ -51,11 +53,14 @@ * * * @author Artem Bilan + * @author Christian Tzolov * * @since 5.0 */ public class MockMessageHandler extends AbstractMessageProducingHandler { + private final Lock lock = new ReentrantLock(); + protected final List, ?>> messageFunctions = new LinkedList<>(); // NOSONAR final private final CapturingMatcher> capturingMatcher; @@ -110,13 +115,17 @@ protected void handleMessageInternal(Message message) { Function, ?> function = this.lastFunction; - synchronized (this) { + this.lock.tryLock(); + try { Iterator, ?>> iterator = this.messageFunctions.iterator(); if (iterator.hasNext()) { function = iterator.next(); iterator.remove(); } } + finally { + this.lock.unlock(); + } Object result = function.apply(message); diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java index ac7f9d3b4ac..fef569ca23d 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java @@ -21,6 +21,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.Lifecycle; import org.springframework.context.SmartLifecycle; @@ -46,6 +48,7 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 4.1 */ @@ -53,6 +56,8 @@ public final class ClientWebSocketContainer extends IntegrationWebSocketContaine private static final int DEFAULT_CONNECTION_TIMEOUT = 10; + private final Lock lock = new ReentrantLock(); + private final WebSocketHttpHeaders headers = new WebSocketHttpHeaders(); private final IntegrationWebSocketConnectionManager connectionManager; @@ -183,12 +188,18 @@ public boolean isRunning() { } @Override - public synchronized void start() { - if (!isRunning()) { - this.clientSession = null; - this.openConnectionException = null; - this.connectionLatch = new CountDownLatch(1); - this.connectionManager.start(); + public void start() { + this.lock.tryLock(); + try { + if (!isRunning()) { + this.clientSession = null; + this.openConnectionException = null; + this.connectionLatch = new CountDownLatch(1); + this.connectionManager.start(); + } + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java index 792e36b4207..5bd4dbe0048 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java @@ -17,6 +17,8 @@ package org.springframework.integration.websocket; import java.util.Arrays; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.Lifecycle; import org.springframework.context.SmartLifecycle; @@ -47,12 +49,15 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 4.1 */ public class ServerWebSocketContainer extends IntegrationWebSocketContainer implements WebSocketConfigurer, SmartLifecycle { + private final Lock lock = new ReentrantLock(); + private final String[] paths; private HandshakeHandler handshakeHandler; @@ -224,9 +229,15 @@ public boolean isRunning() { } @Override - public synchronized void start() { - if (this.handshakeHandler instanceof Lifecycle && !isRunning()) { - ((Lifecycle) this.handshakeHandler).start(); + public void start() { + this.lock.tryLock(); + try { + if (this.handshakeHandler instanceof Lifecycle && !isRunning()) { + ((Lifecycle) this.handshakeHandler).start(); + } + } + finally { + this.lock.unlock(); } } diff --git a/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java b/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java index 53fc14ea801..e59b4673f51 100644 --- a/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java +++ b/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.net.URI; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.xml.transform.TransformerException; @@ -55,9 +57,12 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public abstract class AbstractWebServiceOutboundGateway extends AbstractReplyProducingMessageHandler { + private final Lock lock = new ReentrantLock(); + protected final DefaultUriBuilderFactory uriFactory = new DefaultUriBuilderFactory(); // NOSONAR - final private final String uri; @@ -108,10 +113,14 @@ public void setHeaderMapper(SoapHeaderMapper headerMapper) { * @param uriVariableExpressions The URI variable expressions. */ public void setUriVariableExpressions(Map uriVariableExpressions) { - synchronized (this.uriVariableExpressions) { + this.lock.tryLock(); + try { this.uriVariableExpressions.clear(); this.uriVariableExpressions.putAll(uriVariableExpressions); } + finally { + this.lock.unlock(); + } } /** diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java index 3b7ea3853b8..f9fbb8eb6eb 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.io.File; import java.io.InputStream; import java.io.StringReader; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; @@ -43,11 +45,13 @@ * * @author Jonas Partner * @author Artem Bilan + * @author Christian Tzolov */ public class DefaultXmlPayloadConverter implements XmlPayloadConverter { private final DocumentBuilderFactory documentBuilderFactory; + private final Lock lock = new ReentrantLock(); public DefaultXmlPayloadConverter() { this(DocumentBuilderFactoryUtils.newInstance()); @@ -142,13 +146,17 @@ else if (object instanceof Document) { } } - protected synchronized DocumentBuilder getDocumentBuilder() { + protected DocumentBuilder getDocumentBuilder() { + this.lock.tryLock(); try { return this.documentBuilderFactory.newDocumentBuilder(); } catch (ParserConfigurationException e) { throw new MessagingException("failed to create a new DocumentBuilder", e); } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java index ab2593ce47a..4b7222207e0 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,8 @@ import java.io.File; import java.io.StringReader; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; @@ -39,9 +41,12 @@ * @author Jonas Partner * @author Mark Fisher * @author Artem Bilan + * @author Christian Tzolov */ public class DomSourceFactory implements SourceFactory { + private final Lock lock = new ReentrantLock(); + private final DocumentBuilderFactory documentBuilderFactory; @@ -99,9 +104,13 @@ private DOMSource createDomSourceForFile(File file) { } private DocumentBuilder getNewDocumentBuilder() throws ParserConfigurationException { - synchronized (this.documentBuilderFactory) { + this.lock.tryLock(); + try { return this.documentBuilderFactory.newDocumentBuilder(); } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java index 16599180d02..577a6414c53 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,8 @@ import java.io.File; import java.io.FileReader; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.xml.transform.Source; import javax.xml.transform.Transformer; @@ -40,9 +42,12 @@ * @author Jonas Partner * @author Mark Fisher * @author Artem Bilan + * @author Christian Tzolov */ public class StringSourceFactory implements SourceFactory { + private final Lock lock = new ReentrantLock(); + private final TransformerFactory transformerFactory; @@ -96,13 +101,17 @@ private StringSource createStringSourceForFile(File file) { } } - private synchronized Transformer getTransformer() { + private Transformer getTransformer() { + this.lock.tryLock(); try { return this.transformerFactory.newTransformer(); } catch (Exception e) { throw new MessagingException("Exception creating transformer", e); } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java index 9cdbc40a1c6..a5ab3233768 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.xml.transformer; import java.io.StringReader; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; @@ -38,9 +40,12 @@ * * @author Jonas Partner * @author Artem Bilan + * @author Christian Tzolov */ public class ResultToDocumentTransformer implements ResultTransformer { + private final Lock lock = new ReentrantLock(); + // Not guaranteed to be thread safe private final DocumentBuilderFactory documentBuilderFactory; @@ -84,13 +89,17 @@ private Document createDocumentFromStringResult(StringResult stringResult) { } } - private synchronized DocumentBuilder getDocumentBuilder() { + private DocumentBuilder getDocumentBuilder() { + this.lock.tryLock(); try { return this.documentBuilderFactory.newDocumentBuilder(); } catch (ParserConfigurationException e) { throw new MessagingException("failed to create a new DocumentBuilder", e); } + finally { + this.lock.unlock(); + } } } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java index ee966a2cc97..a788106370c 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.xml.transformer; import java.util.Properties; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.xml.transform.Result; import javax.xml.transform.Transformer; @@ -38,9 +40,12 @@ * @author Jonas Partner * @author Mark Fisher * @author Artem Bilan + * @author Christian Tzolov */ public class ResultToStringTransformer implements ResultTransformer { + private final Lock lock = new ReentrantLock(); + private final TransformerFactory transformerFactory; private Properties outputProperties; @@ -90,9 +95,13 @@ else if (result instanceof DOMResult) { private Transformer getNewTransformer() throws TransformerConfigurationException { Transformer transformer; - synchronized (this.transformerFactory) { + this.lock.tryLock(); + try { transformer = this.transformerFactory.newTransformer(); } + finally { + this.lock.unlock(); + } if (this.outputProperties != null) { transformer.setOutputProperties(this.outputProperties); } diff --git a/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java b/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java index a0a9c4ca9ba..8e46de6945e 100644 --- a/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java +++ b/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2020 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.xmpp.config; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.jivesoftware.smack.ConnectionListener; import org.jivesoftware.smack.XMPPConnection; import org.jivesoftware.smack.roster.Roster; @@ -40,6 +43,7 @@ * @author Artem Bilan * @author Philipp Etschel * @author Gary Russell + * @author Christian Tzolov * * @since 2.0 * @@ -47,7 +51,7 @@ */ public class XmppConnectionFactoryBean extends AbstractFactoryBean implements SmartLifecycle { - private final Object lifecycleMonitor = new Object(); + private final Lock lifecycleMonitor = new ReentrantLock(); private XMPPTCPConnectionConfiguration connectionConfiguration; @@ -172,7 +176,8 @@ protected XMPPTCPConnection getConnection() { @Override public void start() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (this.running) { return; } @@ -195,16 +200,23 @@ public void start() { + connection.getXMPPServiceDomain(), e); } } + finally { + this.lifecycleMonitor.unlock(); + } } @Override public void stop() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (this.isRunning()) { getConnection().disconnect(); this.running = false; } } + finally { + this.lifecycleMonitor.unlock(); + } } @Override diff --git a/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java b/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java index 681f4fd0a86..d6278569d45 100644 --- a/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java +++ b/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java @@ -20,6 +20,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import org.apache.commons.logging.Log; @@ -58,6 +60,7 @@ * The address for this socket is {@code "inproc://" + beanName + ".capture"}. * * @author Artem Bilan + * @author Christian Tzolov * * @since 5.4 * @@ -67,6 +70,8 @@ public class ZeroMqProxy implements InitializingBean, SmartLifecycle, BeanNameAw private static final Log LOG = LogFactory.getLog(ZeroMqProxy.class); + private final Lock lock = new ReentrantLock(); + private final ZContext context; private final Type type; @@ -247,65 +252,78 @@ public void afterPropertiesSet() { } @Override - public synchronized void start() { - if (!this.running.get()) { - this.proxyExecutor - .execute(() -> { - ZMQ.Socket captureSocket = null; - if (this.exposeCaptureSocket) { - captureSocket = this.context.createSocket(SocketType.PUB); - } - try ( - ZMQ.Socket frontendSocket = this.context.createSocket(this.type.getFrontendSocketType()); - ZMQ.Socket backendSocket = this.context.createSocket(this.type.getBackendSocketType()); - ZMQ.Socket controlSocket = this.context.createSocket(SocketType.PAIR) - ) { - - if (this.frontendSocketConfigurer != null) { - this.frontendSocketConfigurer.accept(frontendSocket); + public void start() { + this.lock.tryLock(); + try { + if (!this.running.get()) { + this.proxyExecutor + .execute(() -> { + ZMQ.Socket captureSocket = null; + if (this.exposeCaptureSocket) { + captureSocket = this.context.createSocket(SocketType.PUB); } + try ( + ZMQ.Socket frontendSocket = this.context + .createSocket(this.type.getFrontendSocketType()); + ZMQ.Socket backendSocket = this.context + .createSocket(this.type.getBackendSocketType()); + ZMQ.Socket controlSocket = this.context.createSocket(SocketType.PAIR)) { + + if (this.frontendSocketConfigurer != null) { + this.frontendSocketConfigurer.accept(frontendSocket); + } - if (this.backendSocketConfigurer != null) { - this.backendSocketConfigurer.accept(backendSocket); - } + if (this.backendSocketConfigurer != null) { + this.backendSocketConfigurer.accept(backendSocket); + } - this.frontendPort.set(bindSocket(frontendSocket, this.frontendPort.get())); // NOSONAR - this.backendPort.set(bindSocket(backendSocket, this.backendPort.get())); // NOSONAR - boolean bound = controlSocket.bind(this.controlAddress); // NOSONAR - if (!bound) { - throw new IllegalArgumentException("Cannot bind ZeroMQ socket to address: " - + this.controlAddress); - } - if (captureSocket != null) { - bound = captureSocket.bind(this.captureAddress); + this.frontendPort.set(bindSocket(frontendSocket, this.frontendPort.get())); // NOSONAR + this.backendPort.set(bindSocket(backendSocket, this.backendPort.get())); // NOSONAR + boolean bound = controlSocket.bind(this.controlAddress); // NOSONAR if (!bound) { throw new IllegalArgumentException("Cannot bind ZeroMQ socket to address: " - + this.captureAddress); + + this.controlAddress); + } + if (captureSocket != null) { + bound = captureSocket.bind(this.captureAddress); + if (!bound) { + throw new IllegalArgumentException("Cannot bind ZeroMQ socket to address: " + + this.captureAddress); + } } + this.running.set(true); + ZMQ.proxy(frontendSocket, backendSocket, captureSocket, controlSocket); } - this.running.set(true); - ZMQ.proxy(frontendSocket, backendSocket, captureSocket, controlSocket); - } - catch (Exception ex) { // NOSONAR - LOG.error("Cannot start ZeroMQ proxy from bean: " + this.beanName, ex); - } - finally { - if (captureSocket != null) { - captureSocket.close(); + catch (Exception ex) { // NOSONAR + LOG.error("Cannot start ZeroMQ proxy from bean: " + this.beanName, ex); } - } - }); + finally { + if (captureSocket != null) { + captureSocket.close(); + } + } + }); + } + } + finally { + this.lock.unlock(); } } @Override - public synchronized void stop() { - if (this.running.getAndSet(false)) { - try (ZMQ.Socket commandSocket = this.context.createSocket(SocketType.PAIR)) { - commandSocket.connect(this.controlAddress); // NOSONAR - commandSocket.send(zmq.ZMQ.PROXY_TERMINATE); + public void stop() { + this.lock.tryLock(); + try { + if (this.running.getAndSet(false)) { + try (ZMQ.Socket commandSocket = this.context.createSocket(SocketType.PAIR)) { + commandSocket.connect(this.controlAddress); // NOSONAR + commandSocket.send(zmq.ZMQ.PROXY_TERMINATE); + } } } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java index 933ecf69584..2583dda4089 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2019 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,9 @@ package org.springframework.integration.zookeeper.config; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -31,12 +34,13 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 4.2 */ public class CuratorFrameworkFactoryBean implements FactoryBean, SmartLifecycle { - private final Object lifecycleLock = new Object(); + private final Lock lifecycleLock = new ReentrantLock(); private final CuratorFramework client; @@ -109,7 +113,8 @@ public void setAutoStartup(boolean autoStartup) { @Override public void start() { - synchronized (this.lifecycleLock) { + this.lifecycleLock.tryLock(); + try { if (!this.running) { if (this.client != null) { this.client.start(); @@ -117,16 +122,23 @@ public void start() { this.running = true; } } + finally { + this.lifecycleLock.unlock(); + } } @Override public void stop() { - synchronized (this.lifecycleLock) { + this.lifecycleLock.tryLock(); + try { if (this.running) { CloseableUtils.closeQuietly(this.client); this.running = false; } } + finally { + this.lifecycleLock.unlock(); + } } @Override diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java index 1a098a878b0..4309eede790 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2020 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,6 +17,8 @@ package org.springframework.integration.zookeeper.config; import java.util.UUID; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.curator.framework.CuratorFramework; @@ -37,12 +39,15 @@ * * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 4.2 */ public class LeaderInitiatorFactoryBean implements FactoryBean, SmartLifecycle, InitializingBean, ApplicationEventPublisherAware { + private final Lock lock = new ReentrantLock(); + private CuratorFramework client; private Candidate candidate; @@ -177,8 +182,14 @@ else if (this.applicationEventPublisher != null) { } @Override - public synchronized LeaderInitiator getObject() { - return this.leaderInitiator; + public LeaderInitiator getObject() { + this.lock.tryLock(); + try { + return this.leaderInitiator; + } + finally { + this.lock.unlock(); + } } @Override diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java index 8203d525b26..59c8b69fd8f 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java @@ -18,6 +18,8 @@ import java.util.Collection; import java.util.List; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -43,6 +45,7 @@ * @author Gary Russell * @author Artem Bilan * @author Ivan Zaitsev + * @author Christian Tzolov * * @since 4.2 */ @@ -66,7 +69,7 @@ public class LeaderInitiator implements SmartLifecycle { */ private final Candidate candidate; - private final Object lifecycleMonitor = new Object(); + private final Lock lifecycleMonitor = new ReentrantLock(); /** * Base path in a zookeeper @@ -159,7 +162,8 @@ public void setAutoStartup(boolean autoStartup) { */ @Override public void start() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (!this.running) { if (this.client.getState() != CuratorFrameworkState.STARTED) { // we want to do curator start here because it needs to @@ -177,6 +181,9 @@ public void start() { LOGGER.debug("Started LeaderInitiator"); } } + finally { + this.lifecycleMonitor.unlock(); + } } /** @@ -185,13 +192,17 @@ public void start() { */ @Override public void stop() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.tryLock(); + try { if (this.running) { this.leaderSelector.close(); this.running = false; LOGGER.debug("Stopped LeaderInitiator"); } } + finally { + this.lifecycleMonitor.unlock(); + } } /** diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java index d7afaba8512..a67e01caa1a 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2021 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.locks.InterProcessMutex; @@ -44,6 +45,7 @@ * @author Artem Bilan * @author Vedran Pavic * @author Unseok Kim + * @author Christian Tzolov * * @since 4.2 * @@ -58,6 +60,8 @@ public class ZookeeperLockRegistry implements ExpirableLockRegistry, DisposableB private static final int DEFAULT_CAPACITY = 30_000; + private final Lock locksLock = new ReentrantLock(); + private final Map locks = new LinkedHashMap(16, 0.75F, true) { @@ -145,8 +149,11 @@ public Lock obtain(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = this.keyToPath.pathFor((String) lockKey); ZkLock lock; - synchronized (this.locks) { + this.locksLock.tryLock(); + try { lock = this.locks.computeIfAbsent(path, p -> new ZkLock(this.client, this.mutexTaskExecutor, p)); + } finally { + this.locksLock.unlock(); } if (this.trackingTime) { lock.setLastUsed(System.currentTimeMillis()); @@ -155,10 +162,9 @@ public Lock obtain(Object lockKey) { } /** - * Remove locks last acquired more than 'age' ago that are not currently locked. - * Expiry is not supported if the {@link KeyToPathStrategy} is bounded (returns a finite - * number of paths). With such a {@link KeyToPathStrategy}, the overhead of tracking when - * a lock is obtained is avoided. + * Remove locks last acquired more than 'age' ago that are not currently locked. Expiry is not supported if the + * {@link KeyToPathStrategy} is bounded (returns a finite number of paths). With such a {@link KeyToPathStrategy}, + * the overhead of tracking when a lock is obtained is avoided. * @param age the time since the lock was last obtained. */ @Override @@ -168,13 +174,18 @@ public void expireUnusedOlderThan(long age) { } long now = System.currentTimeMillis(); - synchronized (this.locks) { + this.locksLock.tryLock(); + try { this.locks.entrySet() .removeIf(entry -> { ZkLock lock = entry.getValue(); return now - lock.getLastUsed() > age && !lock.isAcquiredInThisProcess(); }); } + finally { + this.locksLock.unlock(); + } + } @Override diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java index 10cdbbb5e95..f5471519a06 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2022 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; @@ -43,6 +45,7 @@ * @author Marius Bogoevici * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 4.2 */ @@ -72,6 +75,8 @@ public class ZookeeperMetadataStore implements ListenableMetadataStore, SmartLif private boolean running; + private final Lock lock = new ReentrantLock(); + public ZookeeperMetadataStore(CuratorFramework client) { Assert.notNull(client, "Client cannot be null"); this.client = client; // NOSONAR @@ -110,42 +115,54 @@ public void setPhase(int phase) { } @Override - public synchronized String putIfAbsent(String key, String value) { - Assert.notNull(key, KEY_MUST_NOT_BE_NULL); - Assert.notNull(value, "'value' must not be null."); + public String putIfAbsent(String key, String value) { + this.lock.tryLock(); try { - createNode(key, value); - return null; - } - catch (KeeperException.NodeExistsException ex) { - // so the data actually exists, we can read it - return get(key); + Assert.notNull(key, KEY_MUST_NOT_BE_NULL); + Assert.notNull(value, "'value' must not be null."); + try { + createNode(key, value); + return null; + } + catch (KeeperException.NodeExistsException ex) { + // so the data actually exists, we can read it + return get(key); + } + catch (Exception ex) { + throw new ZookeeperMetadataStoreException("Error while trying to set '" + key + "':", ex); + } } - catch (Exception ex) { - throw new ZookeeperMetadataStoreException("Error while trying to set '" + key + "':", ex); + finally { + this.lock.unlock(); } } @Override - public synchronized boolean replace(String key, String oldValue, String newValue) { - Assert.notNull(key, KEY_MUST_NOT_BE_NULL); - Assert.notNull(oldValue, "'oldValue' must not be null."); - Assert.notNull(newValue, "'newValue' must not be null."); - Stat currentStat = new Stat(); + public boolean replace(String key, String oldValue, String newValue) { + this.lock.tryLock(); try { - byte[] bytes = this.client.getData().storingStatIn(currentStat).forPath(getPath(key)); - if (oldValue.equals(IntegrationUtils.bytesToString(bytes, this.encoding))) { - updateNode(key, newValue, currentStat.getVersion()); + Assert.notNull(key, KEY_MUST_NOT_BE_NULL); + Assert.notNull(oldValue, "'oldValue' must not be null."); + Assert.notNull(newValue, "'newValue' must not be null."); + Stat currentStat = new Stat(); + try { + byte[] bytes = this.client.getData().storingStatIn(currentStat).forPath(getPath(key)); + if (oldValue.equals(IntegrationUtils.bytesToString(bytes, this.encoding))) { + updateNode(key, newValue, currentStat.getVersion()); + } + return true; + } + catch (KeeperException.NoNodeException | KeeperException.BadVersionException ex) { + // ignore, the node doesn't exist there's nothing to replace + return false; + } + // ignore + catch (Exception ex) { + throw new ZookeeperMetadataStoreException("Cannot replace value", ex); } - return true; - } - catch (KeeperException.NoNodeException | KeeperException.BadVersionException ex) { - // ignore, the node doesn't exist there's nothing to replace - return false; } - // ignore - catch (Exception ex) { - throw new ZookeeperMetadataStoreException("Cannot replace value", ex); + finally { + this.lock.unlock(); } } @@ -161,71 +178,89 @@ public void removeListener(MetadataStoreListener callback) { } @Override - public synchronized void put(String key, String value) { - Assert.notNull(key, KEY_MUST_NOT_BE_NULL); - Assert.notNull(value, "'value' must not be null."); + public void put(String key, String value) { + this.lock.tryLock(); try { - Stat currentNode = this.client.checkExists().forPath(getPath(key)); - if (currentNode == null) { - try { - createNode(key, value); + Assert.notNull(key, KEY_MUST_NOT_BE_NULL); + Assert.notNull(value, "'value' must not be null."); + try { + Stat currentNode = this.client.checkExists().forPath(getPath(key)); + if (currentNode == null) { + try { + createNode(key, value); + } + catch (KeeperException.NodeExistsException e) { + updateNode(key, value, -1); + } } - catch (KeeperException.NodeExistsException e) { + else { updateNode(key, value, -1); } } - else { - updateNode(key, value, -1); + catch (Exception ex) { + throw new ZookeeperMetadataStoreException("Error while setting value for key '" + key + "':", ex); } } - catch (Exception ex) { - throw new ZookeeperMetadataStoreException("Error while setting value for key '" + key + "':", ex); + finally { + this.lock.unlock(); } } @Override - public synchronized String get(String key) { - Assert.notNull(key, KEY_MUST_NOT_BE_NULL); - Assert.state(isRunning(), "ZookeeperMetadataStore has to be started before using."); - return this.cache.get(getPath(key)) - .map(currentData -> { - // our version is more recent than the cache - if (this.updateMap.containsKey(key) && - this.updateMap.get(key).version() >= currentData.getStat().getVersion()) { - - return this.updateMap.get(key).value(); - } - return IntegrationUtils.bytesToString(currentData.getData(), this.encoding); // NOSONAR - }) - .orElseGet(() -> { - if (this.updateMap.containsKey(key)) { - // we have saved the value, but the cache hasn't updated yet - // if the value had changed via replication, we would have been notified by the listener - return this.updateMap.get(key).value(); - } - else { - // the value just doesn't exist - return null; - } - }); + public String get(String key) { + this.lock.tryLock(); + try { + Assert.notNull(key, KEY_MUST_NOT_BE_NULL); + Assert.state(isRunning(), "ZookeeperMetadataStore has to be started before using."); + return this.cache.get(getPath(key)) + .map(currentData -> { + // our version is more recent than the cache + if (this.updateMap.containsKey(key) && + this.updateMap.get(key).version() >= currentData.getStat().getVersion()) { + + return this.updateMap.get(key).value(); + } + return IntegrationUtils.bytesToString(currentData.getData(), this.encoding); // NOSONAR + }) + .orElseGet(() -> { + if (this.updateMap.containsKey(key)) { + // we have saved the value, but the cache hasn't updated yet + // if the value had changed via replication, we would have been notified by the listener + return this.updateMap.get(key).value(); + } + else { + // the value just doesn't exist + return null; + } + }); + } + finally { + this.lock.unlock(); + } } @Override - public synchronized String remove(String key) { - Assert.notNull(key, KEY_MUST_NOT_BE_NULL); + public String remove(String key) { + this.lock.tryLock(); try { - byte[] bytes = this.client.getData().forPath(getPath(key)); - this.client.delete().forPath(getPath(key)); - // we guarantee that the deletion will supersede the existing data - this.updateMap.put(key, new LocalChildData(null, Integer.MAX_VALUE)); - return IntegrationUtils.bytesToString(bytes, this.encoding); - } - catch (KeeperException.NoNodeException ex) { - // ignore - the node doesn't exist - return null; + Assert.notNull(key, KEY_MUST_NOT_BE_NULL); + try { + byte[] bytes = this.client.getData().forPath(getPath(key)); + this.client.delete().forPath(getPath(key)); + // we guarantee that the deletion will supersede the existing data + this.updateMap.put(key, new LocalChildData(null, Integer.MAX_VALUE)); + return IntegrationUtils.bytesToString(bytes, this.encoding); + } + catch (KeeperException.NoNodeException ex) { + // ignore - the node doesn't exist + return null; + } + catch (Exception ex) { + throw new ZookeeperMetadataStoreException("Exception while deleting key '" + key + "'", ex); + } } - catch (Exception ex) { - throw new ZookeeperMetadataStoreException("Exception while deleting key '" + key + "'", ex); + finally { + this.lock.unlock(); } } @@ -250,35 +285,54 @@ public boolean isAutoStartup() { } @Override - public synchronized void start() { - if (!this.running) { - try { - this.client.createContainers(this.root); - this.cache = CuratorCache.builder(this.client, this.root).build(); - this.cache.listenable().addListener(new MetadataStoreCacheListener()); - this.cache.start(); - this.running = true; - } - catch (Exception ex) { - throw new ZookeeperMetadataStoreException("Exception while starting bean", ex); + public void start() { + this.lock.tryLock(); + try { + if (!this.running) { + try { + this.client.createContainers(this.root); + this.cache = CuratorCache.builder(this.client, this.root).build(); + this.cache.listenable().addListener(new MetadataStoreCacheListener()); + this.cache.start(); + this.running = true; + } + catch (Exception ex) { + throw new ZookeeperMetadataStoreException("Exception while starting bean", ex); + } } + + } + finally { + this.lock.unlock(); } } @Override - public synchronized void stop() { - if (this.running) { - if (this.cache != null) { - CloseableUtils.closeQuietly(this.cache); + public void stop() { + this.lock.tryLock(); + try { + if (this.running) { + if (this.cache != null) { + CloseableUtils.closeQuietly(this.cache); + } + this.cache = null; + this.running = false; } - this.cache = null; - this.running = false; + } + finally { + this.lock.unlock(); } } @Override - public synchronized boolean isRunning() { - return this.running; + public boolean isRunning() { + this.lock.tryLock(); + try { + return this.running; + } + finally { + this.lock.unlock(); + } } @Override From f74c89a8f605773999f514347652d22b598241e6 Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Fri, 16 Jun 2023 16:02:09 +0200 Subject: [PATCH 02/10] fix checkstyle --- .../config/AbstractSimpleMessageHandlerFactoryBean.java | 3 ++- .../integration/config/xml/SpelPropertyAccessorsParser.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java index 1312742274d..d70ac98c8b8 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java @@ -231,7 +231,8 @@ protected final H createHandlerInternal() { .acceptIfCondition(this.handler instanceof Orderable && this.order != null, this.order, theOrder -> ((Orderable) this.handler).setOrder(theOrder)); this.initialized = true; - } finally { + } + finally { this.initializationMonitor.unlock(); } initializingBean(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java b/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java index cbe22790231..48bd0048c8c 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java @@ -41,7 +41,7 @@ * @author Artem Bilan * @author Gary Russell * @author Christian Tzolov - * + * * @since 3.0 */ public class SpelPropertyAccessorsParser implements BeanDefinitionParser { From b1562e5fabc5ca7e6fd0e9ce1b832b827fc72f09 Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Fri, 16 Jun 2023 16:18:50 +0200 Subject: [PATCH 03/10] use blocking lock --- .../outbound/AbstractAmqpOutboundEndpoint.java | 6 +++--- .../channel/AbstractMessageChannel.java | 2 +- .../channel/DefaultHeaderChannelRegistry.java | 8 ++++---- .../AbstractSimpleMessageHandlerFactoryBean.java | 2 +- .../config/ConsumerEndpointFactoryBean.java | 4 ++-- .../config/IdGeneratorConfigurer.java | 2 +- .../IdempotentReceiverAutoProxyCreator.java | 2 +- .../SourcePollingChannelAdapterFactoryBean.java | 2 +- .../config/xml/SpelPropertyAccessorsParser.java | 2 +- .../integration/core/MessagingTemplate.java | 2 +- .../dispatcher/AbstractDispatcher.java | 4 ++-- .../dispatcher/PartitionedDispatcher.java | 2 +- .../endpoint/AbstractPollingEndpoint.java | 2 +- .../endpoint/MethodInvokingMessageSource.java | 2 +- .../endpoint/SourcePollingChannelAdapter.java | 2 +- .../gateway/GatewayMessageHandler.java | 2 +- .../gateway/GatewayProxyFactoryBean.java | 2 +- .../gateway/MessagingGatewaySupport.java | 2 +- .../graph/IntegrationGraphServer.java | 4 ++-- .../AbstractReplyProducingMessageHandler.java | 2 +- .../integration/handler/DelayHandler.java | 2 +- .../integration/handler/MessageHandlerChain.java | 2 +- .../support/MessagingMethodInvokerHelper.java | 2 +- .../history/MessageHistoryConfigurer.java | 4 ++-- .../router/AbstractMessageRouter.java | 2 +- .../selector/MessageSelectorChain.java | 2 +- .../selector/MetadataStoreSelector.java | 2 +- .../store/AbstractMessageGroupStore.java | 2 +- .../store/PersistentMessageGroup.java | 6 +++--- .../integration/store/SimpleMessageGroup.java | 2 +- .../support/SmartLifecycleRoleController.java | 2 +- .../channel/BeanFactoryChannelResolver.java | 2 +- .../leader/LockRegistryLeaderInitiator.java | 4 ++-- .../RoutingSlipHeaderValueMessageProcessor.java | 2 +- .../util/AcceptOnceCollectionFilter.java | 2 +- .../integration/util/SimplePool.java | 10 +++++----- .../integration/core/TimeBasedUUIDGenerator.java | 2 +- .../feed/inbound/FeedEntryMessageSource.java | 4 ++-- .../file/FileWritingMessageHandler.java | 12 ++++++------ .../file/config/FileListFilterFactoryBean.java | 2 +- .../file/filters/AcceptOnceFileListFilter.java | 4 ++-- .../file/filters/CompositeFileListFilter.java | 2 +- .../file/locking/FileChannelCache.java | 2 +- .../remote/session/CachingSessionFactory.java | 4 ++-- .../hazelcast/leader/LeaderInitiator.java | 4 ++-- ...stractHttpRequestExecutingMessageHandler.java | 2 +- ...actInternetProtocolSendingMessageHandler.java | 4 ++-- .../ip/tcp/TcpSendingMessageHandler.java | 4 ++-- .../connection/AbstractConnectionFactory.java | 4 ++-- .../AbstractServerConnectionFactory.java | 2 +- .../CachingClientConnectionFactory.java | 2 +- .../connection/ClientModeConnectionManager.java | 2 +- .../FailoverClientConnectionFactory.java | 4 ++-- .../TcpConnectionInterceptorSupport.java | 2 +- .../ip/tcp/connection/TcpNetConnection.java | 2 +- .../TcpNioClientConnectionFactory.java | 2 +- .../ip/tcp/connection/TcpNioSSLConnection.java | 4 ++-- .../ip/udp/MulticastReceivingChannelAdapter.java | 2 +- .../ip/udp/MulticastSendingMessageHandler.java | 2 +- .../ip/udp/UnicastReceivingChannelAdapter.java | 2 +- .../ip/udp/UnicastSendingMessageHandler.java | 6 +++--- .../ip/tcp/connection/HelloWorldInterceptor.java | 2 +- .../integration/jdbc/StoredProcExecutor.java | 2 +- .../PostgresChannelMessageTableSubscriber.java | 6 +++--- .../integration/jdbc/lock/JdbcLockRegistry.java | 6 +++--- .../integration/jdbc/LockInterceptor.java | 2 +- .../jdbc/lock/JdbcLockRegistryDelegateTests.java | 10 +++++----- .../integration/jpa/test/Consumer.java | 4 ++-- .../mqtt/core/AbstractMqttClientManager.java | 6 +++--- .../mqtt/core/Mqttv3ClientManager.java | 6 +++--- .../mqtt/core/Mqttv5ClientManager.java | 4 ++-- .../MqttPahoMessageDrivenChannelAdapter.java | 6 +++--- .../Mqttv5PahoMessageDrivenChannelAdapter.java | 2 +- .../mqtt/outbound/MqttPahoMessageHandler.java | 4 ++-- .../redis/util/RedisLockRegistry.java | 4 ++-- .../sftp/session/DefaultSftpSessionFactory.java | 2 +- .../integration/sftp/session/SftpSession.java | 4 ++-- .../integration/smb/session/SmbShare.java | 2 +- .../stomp/outbound/StompMessageHandler.java | 2 +- .../stream/ByteStreamReadingMessageSource.java | 2 +- .../CharacterStreamReadingMessageSource.java | 2 +- .../test/mock/MockMessageHandler.java | 2 +- .../websocket/ClientWebSocketContainer.java | 2 +- .../websocket/ServerWebSocketContainer.java | 2 +- .../ws/AbstractWebServiceOutboundGateway.java | 2 +- .../xml/DefaultXmlPayloadConverter.java | 2 +- .../integration/xml/source/DomSourceFactory.java | 2 +- .../xml/source/StringSourceFactory.java | 2 +- .../transformer/ResultToDocumentTransformer.java | 2 +- .../transformer/ResultToStringTransformer.java | 2 +- .../xmpp/config/XmppConnectionFactoryBean.java | 4 ++-- .../integration/zeromq/ZeroMqProxy.java | 4 ++-- .../config/CuratorFrameworkFactoryBean.java | 4 ++-- .../config/LeaderInitiatorFactoryBean.java | 2 +- .../zookeeper/leader/LeaderInitiator.java | 4 ++-- .../zookeeper/lock/ZookeeperLockRegistry.java | 4 ++-- .../metadata/ZookeeperMetadataStore.java | 16 ++++++++-------- 97 files changed, 160 insertions(+), 160 deletions(-) diff --git a/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java b/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java index 1b0b3645490..171c53c4f12 100644 --- a/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java +++ b/spring-integration-amqp/src/main/java/org/springframework/integration/amqp/outbound/AbstractAmqpOutboundEndpoint.java @@ -342,7 +342,7 @@ public void setConfirmTimeout(long confirmTimeout) { } protected final void setConnectionFactory(ConnectionFactory connectionFactory) { - this.lock.tryLock(); + this.lock.lock(); try { this.connectionFactory = connectionFactory; } @@ -499,7 +499,7 @@ protected void endpointInit() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.running) { if (!this.lazyConnect && this.connectionFactory != null) { @@ -545,7 +545,7 @@ private Runnable checkUnconfirmed() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.running) { doStop(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java b/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java index 8b3d5dba3a1..1517392ed36 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/channel/AbstractMessageChannel.java @@ -491,7 +491,7 @@ public ChannelInterceptorList(LogAccessor logger) { } public boolean set(List interceptors) { - this.lock.tryLock(); + this.lock.lock(); try { this.interceptors.clear(); this.size = interceptors.size(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java b/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java index 1089132738e..8efc15dc6c2 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/channel/DefaultHeaderChannelRegistry.java @@ -126,7 +126,7 @@ protected void onInit() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.running) { Assert.notNull(getTaskScheduler(), "a task scheduler is required"); @@ -143,7 +143,7 @@ public void start() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { this.running = false; if (this.reaperScheduledFuture != null) { @@ -218,7 +218,7 @@ public MessageChannel channelNameToChannel(@Nullable String name) { */ @Override public void runReaper() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.reaperScheduledFuture != null) { this.reaperScheduledFuture.cancel(true); @@ -234,7 +234,7 @@ public void runReaper() { @Override public void run() { - this.lock.tryLock(); + this.lock.lock(); try { logger.trace(() -> "Reaper started; channels size=" + this.channels.size()); Iterator> iterator = this.channels.entrySet().iterator(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java index d70ac98c8b8..44ef9096dbe 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/AbstractSimpleMessageHandlerFactoryBean.java @@ -195,7 +195,7 @@ public H getObject() { } protected final H createHandlerInternal() { - this.initializationMonitor.tryLock(); + this.initializationMonitor.lock(); try { if (this.initialized) { // There was a problem when this method was called already diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java index a0290020399..89df271895e 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/ConsumerEndpointFactoryBean.java @@ -130,7 +130,7 @@ public class ConsumerEndpointFactoryBean public void setHandler(Object handler) { Assert.isTrue(handler instanceof MessageHandler || handler instanceof ReactiveMessageHandler, "'handler' must be an instance of 'MessageHandler' or 'ReactiveMessageHandler'"); - this.handlerMonitor.tryLock(); + this.handlerMonitor.lock(); try { Assert.isNull(this.handler, "handler cannot be overridden"); if (handler instanceof ReactiveMessageHandler) { @@ -310,7 +310,7 @@ public Class getObjectType() { } private void initializeEndpoint() { - this.initializationMonitor.tryLock(); + this.initializationMonitor.lock(); try { if (this.initialized) { return; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java b/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java index 57483c43324..b26cfc98b18 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/IdGeneratorConfigurer.java @@ -57,7 +57,7 @@ public final class IdGeneratorConfigurer implements ApplicationListener beanClass, String beanN private void initIdempotentEndpointsIfNecessary() { if (this.idempotentEndpoints == null) { // NOSONAR (inconsistent sync) - this.lock.tryLock(); + this.lock.lock(); try { if (this.idempotentEndpoints == null) { this.idempotentEndpoints = new LinkedHashMap>(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java index 53b71da1f1c..b273a893a71 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/SourcePollingChannelAdapterFactoryBean.java @@ -162,7 +162,7 @@ public Class getObjectType() { } private void initializeAdapter() { - this.initializationMonitor.tryLock(); + this.initializationMonitor.lock(); try { if (this.initialized) { return; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java b/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java index 48bd0048c8c..0e8c9cdf72e 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/config/xml/SpelPropertyAccessorsParser.java @@ -93,7 +93,7 @@ else if (delegate.nodeNameEquals(ele, BeanDefinitionParserDelegate.REF_ELEMENT)) } private void initializeSpelPropertyAccessorRegistrarIfNecessary(ParserContext parserContext) { - this.lock.tryLock(); + this.lock.lock(); try { if (!parserContext.getRegistry() .containsBeanDefinition(IntegrationContextUtils.SPEL_PROPERTY_ACCESSOR_REGISTRAR_BEAN_NAME)) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java b/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java index 1b7f2ac66a9..e4dcca64aad 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/core/MessagingTemplate.java @@ -90,7 +90,7 @@ public void setDefaultChannel(MessageChannel channel) { @Nullable public Message sendAndReceive(MessageChannel destination, Message requestMessage) { if (!this.throwExceptionOnLateReplySet) { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.throwExceptionOnLateReplySet) { IntegrationProperties integrationProperties = IntegrationContextUtils diff --git a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java index d184b05c0b7..f9277568db2 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/AbstractDispatcher.java @@ -83,7 +83,7 @@ protected Set getHandlers() { */ @Override public boolean addHandler(MessageHandler handler) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(handler, "handler must not be null"); Assert.isTrue(this.handlers.size() < this.maxSubscribers, "Maximum subscribers exceeded"); @@ -108,7 +108,7 @@ public boolean addHandler(MessageHandler handler) { */ @Override public boolean removeHandler(MessageHandler handler) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(handler, "handler must not be null"); boolean removed = this.handlers.remove(handler); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java index 44b40a265e4..afb8de28595 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/dispatcher/PartitionedDispatcher.java @@ -158,7 +158,7 @@ public boolean dispatch(Message message) { private void populatedPartitions() { if (this.partitions.isEmpty()) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.partitions.isEmpty()) { Map partitionsToUse = new HashMap<>(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java index 31b6e4eb2d1..ad8acb75365 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/AbstractPollingEndpoint.java @@ -265,7 +265,7 @@ protected void setReceiveMessageSource(Object source) { @Override protected void onInit() { - this.initializationMonitor.tryLock(); + this.initializationMonitor.lock(); try { if (this.initialized) { return; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java index c88303beb9c..f54ee39aaba 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/MethodInvokingMessageSource.java @@ -70,7 +70,7 @@ public String getComponentType() { @Override protected void onInit() { - this.initializationMonitor.tryLock(); + this.initializationMonitor.lock(); try { if (this.initialized) { return; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java index b75d96ef9c3..b76b69e76cf 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/endpoint/SourcePollingChannelAdapter.java @@ -181,7 +181,7 @@ protected void onInit() { public MessageChannel getOutputChannel() { if (this.outputChannelName != null) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.outputChannelName != null) { this.outputChannel = getChannelResolver().resolveDestination(this.outputChannelName); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java index d203ed3b010..76cc64400a7 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayMessageHandler.java @@ -84,7 +84,7 @@ public void setReplyTimeout(Long replyTimeout) { @Override protected Object handleRequestMessage(Message requestMessage) { if (this.exchanger == null) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.exchanger == null) { initialize(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java index c66305ba44c..4e980042932 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/gateway/GatewayProxyFactoryBean.java @@ -458,7 +458,7 @@ public void registerMetricsCaptor(MetricsCaptor metricsCaptorToRegister) { @Override @SuppressWarnings("unchecked") protected void onInit() { - this.initializationMonitor.tryLock(); + this.initializationMonitor.lock(); try { if (this.initialized) { return; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java b/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java index 759cdb40f08..a8cfcbc3448 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/gateway/MessagingGatewaySupport.java @@ -895,7 +895,7 @@ private RuntimeException wrapExceptionIfNecessary(Throwable t, String descriptio protected void registerReplyMessageCorrelatorIfNecessary() { MessageChannel replyChan = getReplyChannel(); if (replyChan != null && this.replyMessageCorrelator == null) { - this.replyMessageCorrelatorMonitor.tryLock(); + this.replyMessageCorrelatorMonitor.lock(); try { if (this.replyMessageCorrelator != null) { return; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java b/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java index 52f78e13a09..1e11a2c8c86 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/graph/IntegrationGraphServer.java @@ -133,7 +133,7 @@ public void onApplicationEvent(ContextRefreshedEvent event) { */ public Graph getGraph() { if (this.graph == null) { // NOSONAR (sync) - this.lock.tryLock(); + this.lock.lock(); try { if (this.graph == null) { buildGraph(); @@ -179,7 +179,7 @@ private T enhance(T node) { } private Graph buildGraph() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.micrometerEnhancer == null && MicrometerMetricsCaptorConfiguration.METER_REGISTRY_PRESENT) { this.micrometerEnhancer = new MicrometerNodeEnhancer(this.applicationContext); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java index 6cec16311b7..3a2086e614d 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/AbstractReplyProducingMessageHandler.java @@ -77,7 +77,7 @@ protected boolean getRequiresReply() { */ public void setAdviceChain(List adviceChain) { Assert.notEmpty(adviceChain, "adviceChain cannot be empty"); - this.lock.tryLock(); + this.lock.lock(); try { this.adviceChain.clear(); this.adviceChain.addAll(adviceChain); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java index 34a5a3614f8..690c1a18a97 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/DelayHandler.java @@ -622,7 +622,7 @@ public int getDelayedMessageCount() { */ @Override public void reschedulePersistedMessages() { - this.lock.tryLock(); + this.lock.lock(); try { MessageGroup messageGroup = this.messageStore.getMessageGroup(this.messageGroupId); try (Stream> messageStream = messageGroup.streamMessages()) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java index e610c06c674..3f08fb285a1 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/MessageHandlerChain.java @@ -104,7 +104,7 @@ public IntegrationPatternType getIntegrationPatternType() { @Override protected void onInit() { super.onInit(); - this.initializationMonitor.tryLock(); + this.initializationMonitor.lock(); try { if (!this.initialized) { Assert.notEmpty(this.handlers, "handler list must not be empty"); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java b/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java index 1a5db72dce5..10ea3db87c6 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/handler/support/MessagingMethodInvokerHelper.java @@ -493,7 +493,7 @@ private Object processInternal(ParametersWrapper parameters) { } private void initialize() { - this.lock.tryLock(); + this.lock.lock(); try { if (isProvidedMessageHandlerFactoryBean()) { LOGGER.trace("Overriding default instance of MessageHandlerMethodFactory with the one provided."); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java b/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java index 938da033abc..18e3a96af8f 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/history/MessageHistoryConfigurer.java @@ -185,7 +185,7 @@ public int getPhase() { @ManagedOperation @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.running) { for (TrackableComponent component : getTrackableComponents(this.beanFactory)) { @@ -202,7 +202,7 @@ public void start() { @ManagedOperation @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.running) { this.currentlyTrackedComponents.forEach(component -> { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java b/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java index a5fe7ac5d52..2a89d13dabb 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/router/AbstractMessageRouter.java @@ -88,7 +88,7 @@ public void setDefaultOutputChannel(MessageChannel defaultOutputChannel) { @Override public MessageChannel getDefaultOutputChannel() { if (this.defaultOutputChannelName != null) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.defaultOutputChannelName != null) { this.defaultOutputChannel = getChannelResolver().resolveDestination(this.defaultOutputChannelName); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java b/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java index 326cf468654..b3faa24eaa4 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/selector/MessageSelectorChain.java @@ -77,7 +77,7 @@ public void add(int index, MessageSelector selector) { */ public void setSelectors(List selectors) { Assert.notEmpty(selectors, "selectors must not be empty"); - this.lock.tryLock(); + this.lock.lock(); try { this.selectors.clear(); this.selectors.addAll(selectors); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java b/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java index bc5b51a4f69..094973a8060 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/selector/MetadataStoreSelector.java @@ -124,7 +124,7 @@ public boolean accept(Message message) { return this.metadataStore.putIfAbsent(key, value) == null; } else { - this.lock.tryLock(); + this.lock.lock(); try { String oldValue = this.metadataStore.get(key); if (oldValue == null) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java b/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java index 4ef214168f4..8496a51bf1a 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/store/AbstractMessageGroupStore.java @@ -128,7 +128,7 @@ public void registerMessageGroupExpiryCallback(MessageGroupCallback callback) { @Override @ManagedOperation public int expireMessageGroups(long timeout) { - this.lock.tryLock(); + this.lock.lock(); try { int count = 0; long threshold = System.currentTimeMillis() - timeout; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java b/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java index 498eac4f388..68e245a4f17 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/store/PersistentMessageGroup.java @@ -81,7 +81,7 @@ public Stream> streamMessages() { @Override public Message getOne() { if (this.oneMessage == null) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.oneMessage == null) { if (LOGGER.isDebugEnabled()) { @@ -118,7 +118,7 @@ public int getSequenceSize() { @Override public int size() { if (this.size == 0) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.size == 0) { if (LOGGER.isDebugEnabled()) { @@ -217,7 +217,7 @@ private final class PersistentCollection extends AbstractCollection> private void load() { if (this.collection == null) { - this.innerLock.tryLock(); + this.innerLock.lock(); try { if (this.collection == null) { Object groupId = PersistentMessageGroup.this.original.getGroupId(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java b/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java index cd5edeeb15b..2b878ef991e 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/store/SimpleMessageGroup.java @@ -194,7 +194,7 @@ public String getCondition() { @Override public Message getOne() { - this.lock.tryLock(); + this.lock.lock(); try { Iterator> iterator = this.messages.iterator(); return iterator.hasNext() ? iterator.next() : null; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java b/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java index 5bfd3052a69..5e4c72d6bae 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/support/SmartLifecycleRoleController.java @@ -289,7 +289,7 @@ public Map getEndpointsRunningStatus(String role) { } private void addLazyLifecycles() { - this.lock.tryLock(); + this.lock.lock(); try { this.lazyLifecycles.forEach(this::doAddLifecyclesToRole); this.lazyLifecycles.clear(); diff --git a/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java b/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java index a1d53e8986e..30428f34e37 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/support/channel/BeanFactoryChannelResolver.java @@ -99,7 +99,7 @@ public MessageChannel resolveDestination(String name) { + name + "' exists, but failed to be created", e); } if (!this.initialized) { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.initialized) { try { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/support/leader/LockRegistryLeaderInitiator.java b/spring-integration-core/src/main/java/org/springframework/integration/support/leader/LockRegistryLeaderInitiator.java index 78c9a6b09b0..a20f0f9d9fd 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/support/leader/LockRegistryLeaderInitiator.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/support/leader/LockRegistryLeaderInitiator.java @@ -291,7 +291,7 @@ public void setPublishFailedEvents(boolean publishFailedEvents) { */ @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.leaderEventPublisher == null && this.applicationEventPublisher != null) { this.leaderEventPublisher = new DefaultLeaderEventPublisher(this.applicationEventPublisher); @@ -319,7 +319,7 @@ public void destroy() { */ @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.running) { this.running = false; diff --git a/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java b/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java index b8d78effa09..e554157f864 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/transformer/support/RoutingSlipHeaderValueMessageProcessor.java @@ -85,7 +85,7 @@ public Map, Integer> processMessage(Message message) { // use a local variable to avoid the second access to volatile field on the happy path Map, Integer> slip = this.routingSlip; if (slip == null) { - this.lock.tryLock(); + this.lock.lock(); try { slip = this.routingSlip; if (slip == null) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java b/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java index 256d1d93e58..26e49a9cff0 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/util/AcceptOnceCollectionFilter.java @@ -41,7 +41,7 @@ public class AcceptOnceCollectionFilter implements CollectionFilter { private final Lock lock = new ReentrantLock(); public Collection filter(Collection unfilteredElements) { - this.lock.tryLock(); + this.lock.lock(); try { List filteredElements = new ArrayList<>(); for (T element : unfilteredElements) { diff --git a/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java b/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java index 44642276e9c..1658ff8c636 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/util/SimplePool.java @@ -99,7 +99,7 @@ public SimplePool(int poolSize, PoolItemCallback callback) { * @param poolSize The desired target pool size. */ public void setPoolSize(int poolSize) { - this.lock.tryLock(); + this.lock.lock(); try { int delta = poolSize - this.poolSize.get(); this.targetPoolSize.addAndGet(delta); @@ -147,7 +147,7 @@ public void setPoolSize(int poolSize) { */ @Override public int getPoolSize() { - this.lock.tryLock(); + this.lock.lock(); try { return this.poolSize.get(); } @@ -242,7 +242,7 @@ else if (this.callback.isStale(item)) { */ @Override public void releaseItem(T item) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(item, "Item cannot be null"); Assert.isTrue(this.allocated.contains(item), @@ -274,7 +274,7 @@ public void releaseItem(T item) { @Override public void removeAllIdleItems() { - this.lock.tryLock(); + this.lock.lock(); try { T item; while ((item = this.available.poll()) != null) { @@ -297,7 +297,7 @@ private void doRemoveItem(T item) { @Override public void close() { - this.lock.tryLock(); + this.lock.lock(); try { this.closed = true; removeAllIdleItems(); diff --git a/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java b/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java index cff04681441..ad0fdf6c5ff 100644 --- a/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java +++ b/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java @@ -59,7 +59,7 @@ static UUID generateId() { static UUID generateIdFromTimestamp(long currentTimeMillis) { long time; - lock.tryLock(); + lock.lock(); try { if (currentTimeMillis > lastTime) { lastTime = currentTimeMillis; diff --git a/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java b/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java index 318d289ebaf..229451fc4ec 100644 --- a/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java +++ b/spring-integration-feed/src/main/java/org/springframework/integration/feed/inbound/FeedEntryMessageSource.java @@ -179,7 +179,7 @@ protected SyndEntry doReceive() { Assert.isTrue(this.initialized, "'FeedEntryReaderMessageSource' must be initialized before it can produce Messages."); SyndEntry nextEntry; - this.monitor.tryLock(); + this.monitor.lock(); try { nextEntry = getNextEntry(); if (nextEntry == null) { @@ -232,7 +232,7 @@ private void populateEntryList() { private SyndFeed getFeed() { try { - this.feedMonitor.tryLock(); + this.feedMonitor.lock(); try { SyndFeed feed = buildSyndFeed(); logger.debug(() -> "Retrieved feed for [" + this + "]"); diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java b/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java index 273bcc48593..66225d26c26 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/FileWritingMessageHandler.java @@ -463,7 +463,7 @@ public void start() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.flushTask != null) { this.flushTask.cancel(true); @@ -883,7 +883,7 @@ private File evaluateDestinationDirectoryExpression(Message message) { private FileState getFileState(File fileToWriteTo, boolean isString) throws FileNotFoundException { - this.lock.tryLock(); + this.lock.lock(); try { FileState state; boolean appendNoFlush = FileExistsMode.APPEND_NO_FLUSH.equals(this.fileExistsMode); @@ -988,7 +988,7 @@ public void flushIfNeeded(MessageFlushPredicate flushPredicate, Message filte private Map findFilesToFlush(MessageFlushPredicate flushPredicate, Message filterMessage) { Map toRemove = new HashMap<>(); - this.lock.tryLock(); + this.lock.lock(); try { Iterator> iterator = this.fileStates.entrySet().iterator(); while (iterator.hasNext()) { @@ -1008,7 +1008,7 @@ private Map findFilesToFlush(MessageFlushPredicate flushPredi private void clearState(final File fileToWriteTo, final FileState state) { if (state != null) { - this.lock.tryLock(); + this.lock.lock(); try { this.fileStates.remove(fileToWriteTo.getAbsolutePath()); } @@ -1037,7 +1037,7 @@ private void doFlush(Map toRemove) { FileWritingMessageHandler.this.logger .debug("Interrupted during flush; not flushed: " + toRestore.keySet()); } - this.lock.tryLock(); + this.lock.lock(); try { for (Entry entry : toRestore.entrySet()) { this.fileStates.putIfAbsent(entry.getKey(), entry.getValue()); @@ -1112,7 +1112,7 @@ private final class Flusher implements Runnable { @Override public void run() { Map toRemove = new HashMap<>(); - FileWritingMessageHandler.this.lock.tryLock(); + FileWritingMessageHandler.this.lock.lock(); try { long expired = FileWritingMessageHandler.this.flushTask == null ? Long.MAX_VALUE : (System.currentTimeMillis() - FileWritingMessageHandler.this.flushInterval); diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java b/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java index e63bdf53a65..8216721a7eb 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/config/FileListFilterFactoryBean.java @@ -99,7 +99,7 @@ public void setAlwaysAcceptDirectories(Boolean alwaysAcceptDirectories) { @NonNull public FileListFilter getObject() { if (this.result == null) { - this.monitor.tryLock(); + this.monitor.lock(); try { this.initializeFileListFilter(); } diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java index c8bae3a8a96..a839ed848de 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/AcceptOnceFileListFilter.java @@ -72,7 +72,7 @@ public AcceptOnceFileListFilter() { @Override public boolean accept(F file) { - this.monitor.tryLock(); + this.monitor.lock(); try { if (this.seenSet.contains(file)) { return false; @@ -92,7 +92,7 @@ public boolean accept(F file) { @Override public void rollback(F file, List files) { - this.monitor.tryLock(); + this.monitor.lock(); try { boolean rollingBack = false; for (F fileToRollback : files) { diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java index 7c4fac7a996..5b10ddbf26f 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/filters/CompositeFileListFilter.java @@ -109,7 +109,7 @@ public final CompositeFileListFilter addFilters(FileListFilter... filters) * @return this CompositeFileListFilter instance with the added filters */ public CompositeFileListFilter addFilters(Collection> filtersToAdd) { - this.lock.tryLock(); + this.lock.lock(); try { for (FileListFilter elf : filtersToAdd) { if (elf instanceof DiscardAwareFileListFilter) { diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java b/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java index adbc63110d9..b52806500f1 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java @@ -76,7 +76,7 @@ public static FileLock tryLockFor(File fileToLock) throws IOException { FileLock lock = null; if (channel != null) { try { - lock = channel.tryLock(); + lock = channel.lock(); } catch (OverlappingFileLockException e) { // File is already locked in this thread or virtual machine diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java b/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java index 44e6604c61c..be98c5f96ea 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/remote/session/CachingSessionFactory.java @@ -152,7 +152,7 @@ public void destroy() { * returned to the cache. */ public void resetCache() { - this.lock.tryLock(); + this.lock.lock(); try { LOGGER.debug("Cache reset; idle sessions will be removed, in-use sessions will be closed when returned"); if (this.isSharedSessionCapable && ((SharedSessionCapable) this.sessionFactory).isSharedSession()) { @@ -198,7 +198,7 @@ private CachedSession(Session targetSession, long sharedSessionEpoch) { @Override public void close() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.released) { diff --git a/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java b/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java index c430c836d67..1db7a909ea5 100644 --- a/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java +++ b/spring-integration-hazelcast/src/main/java/org/springframework/integration/hazelcast/leader/LeaderInitiator.java @@ -214,7 +214,7 @@ public Context getContext() { */ @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.running) { this.leaderSelector = new LeaderSelector(); @@ -239,7 +239,7 @@ public void stop(Runnable callback) { */ @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.running) { this.running = false; diff --git a/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java b/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java index dd0a5891186..e279004c21a 100644 --- a/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java +++ b/spring-integration-http/src/main/java/org/springframework/integration/http/outbound/AbstractHttpRequestExecutingMessageHandler.java @@ -231,7 +231,7 @@ public void setHeaderMapper(HeaderMapper headerMapper) { * @param uriVariableExpressions The URI variable expressions. */ public void setUriVariableExpressions(Map uriVariableExpressions) { - this.lock.tryLock(); + this.lock.lock(); try { this.uriVariableExpressions.clear(); this.uriVariableExpressions.putAll(uriVariableExpressions); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java index 1b19c9829d1..1f9049be2c8 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/AbstractInternetProtocolSendingMessageHandler.java @@ -126,7 +126,7 @@ public int getSoSendBufferSize() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.running) { this.doStart(); @@ -142,7 +142,7 @@ public void start() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.running) { this.doStop(); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java index 0ff890aa6ee..fe33345b7f3 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/TcpSendingMessageHandler.java @@ -254,7 +254,7 @@ protected void onInit() { @Override public void start() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (!this.active) { this.active = true; @@ -284,7 +284,7 @@ public void start() { @Override public void stop() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (this.active) { this.active = false; diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java index a8d7135e67c..5500a369847 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java @@ -549,7 +549,7 @@ protected Executor getTaskExecutor() { if (!this.active) { throw new MessagingException("Connection Factory not started"); } - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (this.taskExecutor == null) { this.privateExecutor = true; @@ -582,7 +582,7 @@ public void stop() { } } } - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (this.privateExecutor) { ExecutorService executorService = (ExecutorService) this.taskExecutor; diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java index 9abfc8b8f42..dba39b669f6 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractServerConnectionFactory.java @@ -76,7 +76,7 @@ public SocketAddress getServerSocketAddress() { @Override public void start() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (!isActive()) { this.setActive(true); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java index 39422ec0a11..f8ef6f00c29 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/CachingClientConnectionFactory.java @@ -392,7 +392,7 @@ public void start() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { this.targetConnectionFactory.stop(); this.pool.removeAllIdleItems(); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java index 51d31a72664..be92fce3970 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/ClientModeConnectionManager.java @@ -59,7 +59,7 @@ public ClientModeConnectionManager( @Override public void run() { - this.lock.tryLock(); + this.lock.lock(); try { try { TcpConnection connection = this.clientConnectionFactory.getConnection(); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java index 406dc216853..3bb492fa023 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/FailoverClientConnectionFactory.java @@ -264,7 +264,7 @@ void incrementEpoch() { * @throws InterruptedException if interrupted. */ private void findAConnection() throws InterruptedException { - this.lock.tryLock(); + this.lock.lock(); try { boolean success = false; AbstractClientConnectionFactory lastFactoryToTry = this.currentFactory; @@ -328,7 +328,7 @@ public boolean isOpen() { */ @Override public void send(Message message) { - this.lock.tryLock(); + this.lock.lock(); try { boolean success = false; AbstractClientConnectionFactory lastFactoryToTry = this.currentFactory; diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java index 03935cf3e87..08a8d8539fd 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpConnectionInterceptorSupport.java @@ -244,7 +244,7 @@ public void addNewConnection(TcpConnection connection) { @Override public void removeDeadConnection(TcpConnection connection) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.removed) { return; diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java index c117597c3f2..22e16e0c6b6 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNetConnection.java @@ -108,7 +108,7 @@ public boolean isOpen() { @Override @SuppressWarnings("unchecked") public void send(Message message) { - this.lock.tryLock(); + this.lock.lock(); try { if (this.socketOutputStream == null) { int writeBufferSize = this.socket.getSendBufferSize(); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java index 5fecf63918e..446d335f38d 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioClientConnectionFactory.java @@ -171,7 +171,7 @@ public void stop() { @Override public void start() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (!isActive()) { setActive(true); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java index 95f5e2d040f..0486f9dd43c 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioSSLConnection.java @@ -288,7 +288,7 @@ private void initializeEngine() { @Override protected ChannelOutputStream getChannelOutputStream() { - this.monitorLock.tryLock(); + this.monitorLock.lock(); try { if (this.sslChannelOutputStream == null) { this.sslChannelOutputStream = new SSLChannelOutputStream(super.getChannelOutputStream()); @@ -341,7 +341,7 @@ final class SSLChannelOutputStream extends ChannelOutputStream { */ @Override protected void doWrite(ByteBuffer plainText) throws IOException { - this.lock.tryLock(); + this.lock.lock(); try { TcpNioSSLConnection.this.writerActive = true; int remaining = plainText.remaining(); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java index 37bdb9ba544..0af8c6d3ecd 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java @@ -70,7 +70,7 @@ public MulticastReceivingChannelAdapter(String group, int port, boolean lengthCh @Override public DatagramSocket getSocket() { - this.lock.tryLock(); + this.lock.lock(); try { if (getTheSocket() == null) { try { diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java index 47af8896029..1b5b0a19536 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java @@ -135,7 +135,7 @@ public MulticastSendingMessageHandler(String destinationExpression) { @Override protected DatagramSocket getSocket() throws IOException { - this.lock.tryLock(); + this.lock.lock(); try { if (getTheSocket() == null) { createSocket(); diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java index 2497f0aa22b..4569390329f 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java @@ -254,7 +254,7 @@ protected DatagramSocket getTheSocket() { } public DatagramSocket getSocket() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.socket == null) { try { diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java index 44d3425649f..77a4f757514 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java @@ -321,7 +321,7 @@ public void handleMessageInternal(Message message) { public void startAckThread() { if (!this.ackThreadRunning) { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.ackThreadRunning) { try { @@ -396,7 +396,7 @@ protected DatagramSocket getTheSocket() { } protected DatagramSocket getSocket() throws IOException { - this.lock.tryLock(); + this.lock.lock(); try { if (this.socket == null) { if (this.acknowledge) { @@ -440,7 +440,7 @@ public void setSoReceiveBufferSize(int size) { @Override public void setLocalAddress(String localAddress) { - this.lock.tryLock(); + this.lock.lock(); try { this.localAddress = localAddress; } diff --git a/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java b/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java index 233da519634..3b16447cf2d 100644 --- a/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java +++ b/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java @@ -63,7 +63,7 @@ public HelloWorldInterceptor(String hello, String world, ApplicationEventPublish @Override public boolean onMessage(Message message) { if (!this.negotiated) { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.negotiated) { Object payload = message.getPayload(); diff --git a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java index f1fa7798f55..7b2b95e2cbf 100644 --- a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java +++ b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/StoredProcExecutor.java @@ -304,7 +304,7 @@ private Map executeStoredProcedureInternal(Object input, String private SimpleJdbcCallOperations obtainSimpleJdbcCall(String storedProcedureName) { SimpleJdbcCallOperations operations = this.jdbcCallOperationsMap.get(storedProcedureName); if (operations == null) { - this.jdbcCallOperationsMapMonitor.tryLock(); + this.jdbcCallOperationsMapMonitor.lock(); try { operations = this.jdbcCallOperationsMap.computeIfAbsent(storedProcedureName, this::createSimpleJdbcCall); diff --git a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java index 5c9359bd0aa..fceb1381e3a 100644 --- a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java +++ b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java @@ -117,7 +117,7 @@ public PostgresChannelMessageTableSubscriber(PgConnectionSupplier connectionSupp */ @Deprecated(since = "6.2", forRemoval = true) public void setExecutor(ExecutorService executor) { - this.lock.tryLock(); + this.lock.lock(); try { setTaskExecutor(new TaskExecutorAdapter(executor)); } @@ -161,7 +161,7 @@ public boolean unsubscribe(Subscription subscription) { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.latch.getCount() > 0) { return; @@ -257,7 +257,7 @@ private boolean isActive() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.future.isDone()) { return; diff --git a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java index 85d434dab4f..215f89875cc 100644 --- a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java +++ b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/lock/JdbcLockRegistry.java @@ -114,7 +114,7 @@ public void setCacheCapacity(int cacheCapacity) { public Lock obtain(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = pathFor((String) lockKey); - this.lock.tryLock(); + this.lock.lock(); try { return this.locks.computeIfAbsent(path, key -> new JdbcLock(this.client, this.idleBetweenTries, key)); } @@ -130,7 +130,7 @@ private String pathFor(String input) { @Override public void expireUnusedOlderThan(long age) { long now = System.currentTimeMillis(); - this.lock.tryLock(); + this.lock.lock(); try { this.locks.entrySet() .removeIf(entry -> { @@ -148,7 +148,7 @@ public void renewLock(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = pathFor((String) lockKey); JdbcLock jdbcLock; - this.lock.tryLock(); + this.lock.lock(); try { jdbcLock = this.locks.get(path); } diff --git a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java index 20cd5957a5c..e181b7a6742 100644 --- a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java +++ b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java @@ -34,7 +34,7 @@ public class LockInterceptor extends ReentrantLock implements MethodInterceptor private final Lock lock = new ReentrantLock(); public Object invoke(MethodInvocation invocation) throws Throwable { - this.lock.tryLock(); + this.lock.lock(); try { return invocation.proceed(); } diff --git a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java index 2ff6a720d8c..20abbd88de6 100644 --- a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java +++ b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java @@ -63,7 +63,7 @@ public void testLessAmountOfUnlockThanLock() { final Lock lock = registry.obtain("foo"); for (int i = 0; i < lockCount; i++) { - lock.tryLock(); + lock.lock(); } for (int i = 0; i < unlockCount; i++) { lock.unlock(); @@ -79,7 +79,7 @@ public void testSameAmountOfUnlockThanLock() { final Lock lock = registry.obtain("foo"); for (int i = 0; i < lockCount; i++) { - lock.tryLock(); + lock.lock(); } for (int i = 0; i < lockCount; i++) { lock.unlock(); @@ -91,7 +91,7 @@ public void testSameAmountOfUnlockThanLock() { @Test public void testTransientDataAccessException() { final Lock lock = registry.obtain("foo"); - lock.tryLock(); + lock.lock(); final AtomicBoolean shouldThrow = new AtomicBoolean(true); doAnswer(invocation -> { @@ -109,7 +109,7 @@ public void testTransientDataAccessException() { @Test public void testTransactionTimedOutException() { final Lock lock = registry.obtain("foo"); - lock.tryLock(); + lock.lock(); final AtomicBoolean shouldThrow = new AtomicBoolean(true); doAnswer(invocation -> { @@ -127,7 +127,7 @@ public void testTransactionTimedOutException() { @Test public void testTransactionSystemException() { final Lock lock = registry.obtain("foo"); - lock.tryLock(); + lock.lock(); final AtomicBoolean shouldThrow = new AtomicBoolean(true); doAnswer(invocation -> { diff --git a/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java b/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java index 5d018b7675d..3238bcf2ca9 100644 --- a/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java +++ b/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java @@ -45,7 +45,7 @@ public final class Consumer { private final Lock lock = new ReentrantLock(); public void receive(Message> message) { - this.lock.tryLock(); + this.lock.lock(); try { logger.info("Service Activator received Message: " + message); MESSAGES.add(message); @@ -56,7 +56,7 @@ public void receive(Message> message) { } public Message> poll(long timeoutInMillis) throws InterruptedException { - this.lock.tryLock(); + this.lock.lock(); try { return MESSAGES.poll(timeoutInMillis, TimeUnit.MILLISECONDS); } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java index 1b176554677..61446eebc62 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java @@ -98,7 +98,7 @@ protected ApplicationEventPublisher getApplicationEventPublisher() { } protected void setClient(T client) { - this.lock.tryLock(); + this.lock.lock(); try { this.client = client; } @@ -146,7 +146,7 @@ public boolean isManualAcks() { @Override public T getClient() { - this.lock.tryLock(); + this.lock.lock(); try { return this.client; } @@ -195,7 +195,7 @@ public boolean removeCallback(ConnectCallback connectCallback) { } public boolean isRunning() { - this.lock.tryLock(); + this.lock.lock(); try { return this.client != null; } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java index 86ef0328450..be77b94cf7b 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java @@ -94,7 +94,7 @@ public MqttConnectOptions getConnectionInfo() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { var client = getClient(); if (client == null) { @@ -146,7 +146,7 @@ private IMqttAsyncClient createClient() throws MqttException { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { var client = getClient(); if (client == null) { @@ -175,7 +175,7 @@ public void stop() { @Override public void connectionLost(Throwable cause) { - this.lock.tryLock(); + this.lock.lock(); try { logger.error("Connection lost, client_id=" + getClientId(), cause); } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java index 167a26d8d1d..2e3cad6e61d 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java @@ -96,7 +96,7 @@ public MqttConnectionOptions getConnectionInfo() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { var client = getClient(); if (client == null) { @@ -147,7 +147,7 @@ private MqttAsyncClient createClient() throws MqttException { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { var client = getClient(); if (client == null) { diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java index 71899964556..a960d2f4efb 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java @@ -185,7 +185,7 @@ protected void doStart() { @SuppressWarnings("deprecation") private void connect() throws MqttException { - this.lock.tryLock(); + this.lock.lock(); try { MqttConnectOptions connectionOptions = this.clientFactory.getConnectionOptions(); var clientManager = getClientManager(); @@ -208,7 +208,7 @@ private void connect() throws MqttException { @Override protected void doStop() { - this.lock.tryLock(); + this.lock.lock(); try { this.readyToSubscribeOnStart = false; try { @@ -340,7 +340,7 @@ private void warnInvalidQosForSubscription(String[] topics, int[] requestedQos, @Override public void connectionLost(Throwable cause) { - this.lock.tryLock(); + this.lock.lock(); try { if (isRunning()) { this.logger.error(() -> "Lost connection: " + cause.getMessage()); diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java index 65c9eccd2e2..988b094eeb0 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java @@ -217,7 +217,7 @@ protected void doStart() { } private void connect() throws MqttException { - this.lock.tryLock(); + this.lock.lock(); try { var clientManager = getClientManager(); if (clientManager == null) { diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java index a720cb83d0c..b5be9fbdbb8 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java @@ -191,7 +191,7 @@ protected void doStop() { } private IMqttAsyncClient checkConnection() throws MqttException { - this.lock.tryLock(); + this.lock.lock(); try { var theClientManager = getClientManager(); if (theClientManager != null) { @@ -265,7 +265,7 @@ private void sendDeliveryComplete(IMqttDeliveryToken token) { @Override public void connectionLost(Throwable cause) { - this.lock.tryLock(); + this.lock.lock(); try { logger.error("Lost connection; will attempt reconnect on next request"); if (this.client != null) { diff --git a/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java b/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java index d447a41d8e9..828c18701d5 100644 --- a/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java +++ b/spring-integration-redis/src/main/java/org/springframework/integration/redis/util/RedisLockRegistry.java @@ -227,7 +227,7 @@ public void setRedisLockType(RedisLockType redisLockType) { public Lock obtain(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = (String) lockKey; - this.lock.tryLock(); + this.lock.lock(); try { return this.locks.computeIfAbsent(path, getRedisLockConstructor(this.redisLockType)); } @@ -239,7 +239,7 @@ public Lock obtain(Object lockKey) { @Override public void expireUnusedOlderThan(long age) { long now = System.currentTimeMillis(); - this.lock.tryLock(); + this.lock.lock(); try { this.locks.entrySet() .removeIf(entry -> { diff --git a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java index 11dbd83794c..782223a87e8 100644 --- a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java +++ b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/DefaultSftpSessionFactory.java @@ -326,7 +326,7 @@ private ClientSession initClientSession() throws IOException { private void initClient() throws IOException { if (!this.initialized) { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.initialized) { doInitClient(); diff --git a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java index 950a6a3ee2e..e26772c418e 100644 --- a/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java +++ b/spring-integration-sftp/src/main/java/org/springframework/integration/sftp/session/SftpSession.java @@ -127,7 +127,7 @@ public boolean finalizeRaw() { @Override public void write(InputStream inputStream, String destination) throws IOException { - this.lock.tryLock(); + this.lock.lock(); try { OutputStream outputStream = this.sftpClient.write(destination); FileCopyUtils.copy(inputStream, outputStream); @@ -139,7 +139,7 @@ public void write(InputStream inputStream, String destination) throws IOExceptio @Override public void append(InputStream inputStream, String destination) throws IOException { - this.lock.tryLock(); + this.lock.lock(); try { OutputStream outputStream = this.sftpClient.write(destination, diff --git a/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java b/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java index 05f12e43def..e3cb4bb9d12 100644 --- a/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java +++ b/spring-integration-smb/src/main/java/org/springframework/integration/smb/session/SmbShare.java @@ -132,7 +132,7 @@ boolean isOpened() { @Override public void close() { - this.lock.tryLock(); + this.lock.lock(); try { this.open.set(false); if (this.closeContext.get()) { diff --git a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java index 48bd346de70..9f017d8926e 100644 --- a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java +++ b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/outbound/StompMessageHandler.java @@ -183,7 +183,7 @@ protected void handleMessageInternal(final Message message) { } private void connectIfNecessary() throws InterruptedException { - this.lock.tryLock(); + this.lock.lock(); try { if (this.stompSession == null || !this.stompSessionManager.isConnected()) { this.stompSessionManager.disconnect(this.sessionHandler); diff --git a/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java b/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java index 6e6ddf17278..edeaef71475 100644 --- a/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java +++ b/spring-integration-stream/src/main/java/org/springframework/integration/stream/ByteStreamReadingMessageSource.java @@ -78,7 +78,7 @@ protected byte[] doReceive() { try { byte[] bytes; int bytesRead = 0; - this.lock.tryLock(); + this.lock.lock(); try { if (this.stream.available() == 0) { return null; diff --git a/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java b/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java index 43fde4da39a..b577d20f559 100644 --- a/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java +++ b/spring-integration-stream/src/main/java/org/springframework/integration/stream/CharacterStreamReadingMessageSource.java @@ -117,7 +117,7 @@ public String getComponentType() { @Override public String doReceive() { try { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.blockToDetectEOF && !this.reader.ready()) { return null; diff --git a/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java b/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java index 6310e21377a..ab2a5d931b6 100644 --- a/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java +++ b/spring-integration-test/src/main/java/org/springframework/integration/test/mock/MockMessageHandler.java @@ -115,7 +115,7 @@ protected void handleMessageInternal(Message message) { Function, ?> function = this.lastFunction; - this.lock.tryLock(); + this.lock.lock(); try { Iterator, ?>> iterator = this.messageFunctions.iterator(); if (iterator.hasNext()) { diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java index fef569ca23d..11403648f04 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java @@ -189,7 +189,7 @@ public boolean isRunning() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!isRunning()) { this.clientSession = null; diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java index 5bd4dbe0048..ee5d7b9614d 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java @@ -230,7 +230,7 @@ public boolean isRunning() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.handshakeHandler instanceof Lifecycle && !isRunning()) { ((Lifecycle) this.handshakeHandler).start(); diff --git a/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java b/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java index e59b4673f51..50f8338b9e7 100644 --- a/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java +++ b/spring-integration-ws/src/main/java/org/springframework/integration/ws/AbstractWebServiceOutboundGateway.java @@ -113,7 +113,7 @@ public void setHeaderMapper(SoapHeaderMapper headerMapper) { * @param uriVariableExpressions The URI variable expressions. */ public void setUriVariableExpressions(Map uriVariableExpressions) { - this.lock.tryLock(); + this.lock.lock(); try { this.uriVariableExpressions.clear(); this.uriVariableExpressions.putAll(uriVariableExpressions); diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java index f9fbb8eb6eb..be48e562bdb 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/DefaultXmlPayloadConverter.java @@ -147,7 +147,7 @@ else if (object instanceof Document) { } protected DocumentBuilder getDocumentBuilder() { - this.lock.tryLock(); + this.lock.lock(); try { return this.documentBuilderFactory.newDocumentBuilder(); } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java index 4b7222207e0..4714e1bbfc2 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/DomSourceFactory.java @@ -104,7 +104,7 @@ private DOMSource createDomSourceForFile(File file) { } private DocumentBuilder getNewDocumentBuilder() throws ParserConfigurationException { - this.lock.tryLock(); + this.lock.lock(); try { return this.documentBuilderFactory.newDocumentBuilder(); } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java index 577a6414c53..b1317fdc1ba 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/source/StringSourceFactory.java @@ -102,7 +102,7 @@ private StringSource createStringSourceForFile(File file) { } private Transformer getTransformer() { - this.lock.tryLock(); + this.lock.lock(); try { return this.transformerFactory.newTransformer(); } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java index a5ab3233768..10a77624bb0 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToDocumentTransformer.java @@ -90,7 +90,7 @@ private Document createDocumentFromStringResult(StringResult stringResult) { } private DocumentBuilder getDocumentBuilder() { - this.lock.tryLock(); + this.lock.lock(); try { return this.documentBuilderFactory.newDocumentBuilder(); } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java index a788106370c..cffdd198969 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/transformer/ResultToStringTransformer.java @@ -95,7 +95,7 @@ else if (result instanceof DOMResult) { private Transformer getNewTransformer() throws TransformerConfigurationException { Transformer transformer; - this.lock.tryLock(); + this.lock.lock(); try { transformer = this.transformerFactory.newTransformer(); } diff --git a/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java b/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java index 8e46de6945e..833bfc7162a 100644 --- a/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java +++ b/spring-integration-xmpp/src/main/java/org/springframework/integration/xmpp/config/XmppConnectionFactoryBean.java @@ -176,7 +176,7 @@ protected XMPPTCPConnection getConnection() { @Override public void start() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (this.running) { return; @@ -207,7 +207,7 @@ public void start() { @Override public void stop() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (this.isRunning()) { getConnection().disconnect(); diff --git a/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java b/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java index d6278569d45..2ed9aea6fdf 100644 --- a/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java +++ b/spring-integration-zeromq/src/main/java/org/springframework/integration/zeromq/ZeroMqProxy.java @@ -253,7 +253,7 @@ public void afterPropertiesSet() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.running.get()) { this.proxyExecutor @@ -312,7 +312,7 @@ public void start() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.running.getAndSet(false)) { try (ZMQ.Socket commandSocket = this.context.createSocket(SocketType.PAIR)) { diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java index 2583dda4089..4fca3c612ea 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/CuratorFrameworkFactoryBean.java @@ -113,7 +113,7 @@ public void setAutoStartup(boolean autoStartup) { @Override public void start() { - this.lifecycleLock.tryLock(); + this.lifecycleLock.lock(); try { if (!this.running) { if (this.client != null) { @@ -129,7 +129,7 @@ public void start() { @Override public void stop() { - this.lifecycleLock.tryLock(); + this.lifecycleLock.lock(); try { if (this.running) { CloseableUtils.closeQuietly(this.client); diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java index 4309eede790..a370c7c0b51 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java @@ -183,7 +183,7 @@ else if (this.applicationEventPublisher != null) { @Override public LeaderInitiator getObject() { - this.lock.tryLock(); + this.lock.lock(); try { return this.leaderInitiator; } diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java index 59c8b69fd8f..b1ae7b11065 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/leader/LeaderInitiator.java @@ -162,7 +162,7 @@ public void setAutoStartup(boolean autoStartup) { */ @Override public void start() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (!this.running) { if (this.client.getState() != CuratorFrameworkState.STARTED) { @@ -192,7 +192,7 @@ public void start() { */ @Override public void stop() { - this.lifecycleMonitor.tryLock(); + this.lifecycleMonitor.lock(); try { if (this.running) { this.leaderSelector.close(); diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java index a67e01caa1a..574cf97080e 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java @@ -149,7 +149,7 @@ public Lock obtain(Object lockKey) { Assert.isInstanceOf(String.class, lockKey); String path = this.keyToPath.pathFor((String) lockKey); ZkLock lock; - this.locksLock.tryLock(); + this.locksLock.lock(); try { lock = this.locks.computeIfAbsent(path, p -> new ZkLock(this.client, this.mutexTaskExecutor, p)); } finally { @@ -174,7 +174,7 @@ public void expireUnusedOlderThan(long age) { } long now = System.currentTimeMillis(); - this.locksLock.tryLock(); + this.locksLock.lock(); try { this.locks.entrySet() .removeIf(entry -> { diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java index f5471519a06..5d944acdfd3 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/metadata/ZookeeperMetadataStore.java @@ -116,7 +116,7 @@ public void setPhase(int phase) { @Override public String putIfAbsent(String key, String value) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(key, KEY_MUST_NOT_BE_NULL); Assert.notNull(value, "'value' must not be null."); @@ -139,7 +139,7 @@ public String putIfAbsent(String key, String value) { @Override public boolean replace(String key, String oldValue, String newValue) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(key, KEY_MUST_NOT_BE_NULL); Assert.notNull(oldValue, "'oldValue' must not be null."); @@ -179,7 +179,7 @@ public void removeListener(MetadataStoreListener callback) { @Override public void put(String key, String value) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(key, KEY_MUST_NOT_BE_NULL); Assert.notNull(value, "'value' must not be null."); @@ -208,7 +208,7 @@ public void put(String key, String value) { @Override public String get(String key) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(key, KEY_MUST_NOT_BE_NULL); Assert.state(isRunning(), "ZookeeperMetadataStore has to be started before using."); @@ -241,7 +241,7 @@ public String get(String key) { @Override public String remove(String key) { - this.lock.tryLock(); + this.lock.lock(); try { Assert.notNull(key, KEY_MUST_NOT_BE_NULL); try { @@ -286,7 +286,7 @@ public boolean isAutoStartup() { @Override public void start() { - this.lock.tryLock(); + this.lock.lock(); try { if (!this.running) { try { @@ -309,7 +309,7 @@ public void start() { @Override public void stop() { - this.lock.tryLock(); + this.lock.lock(); try { if (this.running) { if (this.cache != null) { @@ -326,7 +326,7 @@ public void stop() { @Override public boolean isRunning() { - this.lock.tryLock(); + this.lock.lock(); try { return this.running; } From 8323edaed83fbeffe4889f382f6e48227eeab9ef Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Fri, 16 Jun 2023 17:55:20 +0200 Subject: [PATCH 04/10] Secon pass - handle multi-lock cases --- ...oadableResourceBundleExpressionSource.java | 41 +++- .../connection/AbstractConnectionFactory.java | 27 ++- .../ip/tcp/connection/TcpNioConnection.java | 182 +++++++++++------- .../integration/jms/JmsOutboundGateway.java | 51 ++++- .../kafka/inbound/KafkaMessageSource.java | 143 ++++++++++---- .../stomp/AbstractStompSessionManager.java | 112 +++++++---- .../xml/result/DomResultFactory.java | 19 +- .../xml/splitter/XPathMessageSplitter.java | 18 +- 8 files changed, 422 insertions(+), 171 deletions(-) diff --git a/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java b/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java index a9957ca0954..d065bf1033f 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,6 +25,8 @@ import java.util.Locale; import java.util.Map; import java.util.Properties; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -80,16 +82,22 @@ public class ReloadableResourceBundleExpressionSource implements ExpressionSourc */ private final Map>> cachedFilenames = new HashMap<>(); + private final Lock cachedFilenamesMonitor = new ReentrantLock(); + /** * Cache to hold already loaded properties per filename. */ private final Map cachedProperties = new HashMap<>(); + private final Lock cachedPropertiesMonitor = new ReentrantLock(); + /** * Cache to hold merged loaded properties per locale. */ private final Map cachedMergedProperties = new HashMap<>(); + private final Lock cachedMergedPropertiesMonitor = new ReentrantLock(); + private final ExpressionParser parser = new SpelExpressionParser(new SpelParserConfiguration(true, true)); private String[] basenames = {}; @@ -282,7 +290,8 @@ private String getExpressionString(String key, Locale locale) { * cached forever. */ private PropertiesHolder getMergedProperties(Locale locale) { - synchronized (this.cachedMergedProperties) { + this.cachedMergedPropertiesMonitor.lock(); + try { PropertiesHolder mergedHolder = this.cachedMergedProperties.get(locale); if (mergedHolder != null) { return mergedHolder; @@ -303,6 +312,9 @@ private PropertiesHolder getMergedProperties(Locale locale) { this.cachedMergedProperties.put(locale, mergedHolder); return mergedHolder; } + finally { + this.cachedMergedPropertiesMonitor.unlock(); + } } /** @@ -316,7 +328,8 @@ private PropertiesHolder getMergedProperties(Locale locale) { * @see #calculateFilenamesForLocale */ private List calculateAllFilenames(String basename, Locale locale) { - synchronized (this.cachedFilenames) { + this.cachedFilenamesMonitor.lock(); + try { Map> localeMap = this.cachedFilenames.get(basename); if (localeMap != null) { List filenames = localeMap.get(locale); @@ -345,6 +358,9 @@ private List calculateAllFilenames(String basename, Locale locale) { } return filenames; } + finally { + this.cachedFilenamesMonitor.unlock(); + } } /** @@ -392,7 +408,8 @@ private List calculateFilenamesForLocale(String basename, Locale locale) * @return the current PropertiesHolder for the bundle */ private PropertiesHolder getProperties(String filename) { - synchronized (this.cachedProperties) { + this.cachedPropertiesMonitor.lock(); + try { PropertiesHolder propHolder = this.cachedProperties.get(filename); if (propHolder != null && (propHolder.getRefreshTimestamp() < 0 || @@ -401,6 +418,9 @@ private PropertiesHolder getProperties(String filename) { } return refreshProperties(filename, propHolder); } + finally { + this.cachedPropertiesMonitor.unlock(); + } } /** @@ -539,12 +559,21 @@ private void loadFromProperties(Resource resource, String filename, InputStream */ public void clearCache() { LOGGER.debug("Clearing entire resource bundle cache"); - synchronized (this.cachedProperties) { + this.cachedPropertiesMonitor.lock(); + try { this.cachedProperties.clear(); } - synchronized (this.cachedMergedProperties) { + finally { + this.cachedPropertiesMonitor.unlock(); + } + + this.cachedMergedPropertiesMonitor.lock(); + try { this.cachedMergedProperties.clear(); } + finally { + this.cachedMergedPropertiesMonitor.unlock(); + } } @Override diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java index 5500a369847..55c6da1f631 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/AbstractConnectionFactory.java @@ -80,6 +80,8 @@ public abstract class AbstractConnectionFactory extends IntegrationObjectSupport private final Map connections = new ConcurrentHashMap<>(); + private final Lock connectionsMonitor = new ReentrantLock(); + private final BlockingQueue delayedReads = new LinkedBlockingQueue<>(); private final List senders = Collections.synchronizedList(new ArrayList<>()); @@ -568,7 +570,8 @@ protected Executor getTaskExecutor() { @Override public void stop() { this.active = false; - synchronized (this.connections) { + this.connectionsMonitor.lock(); + try { Iterator> iterator = this.connections.entrySet().iterator(); while (iterator.hasNext()) { TcpConnectionSupport connection = iterator.next().getValue(); @@ -582,6 +585,10 @@ public void stop() { } } } + finally { + this.connectionsMonitor.unlock(); + } + this.lifecycleMonitor.lock(); try { if (this.privateExecutor) { @@ -860,7 +867,8 @@ protected void doAccept(final Selector selector, ServerSocketChannel server, lon } protected void addConnection(TcpConnectionSupport connection) { - synchronized (this.connections) { + this.connectionsMonitor.lock(); + try { if (!this.active) { connection.close(); return; @@ -868,6 +876,9 @@ protected void addConnection(TcpConnectionSupport connection) { this.connections.put(connection.getConnectionId(), connection); logger.debug(() -> getComponentName() + ": Added new connection: " + connection.getConnectionId()); } + finally { + this.connectionsMonitor.unlock(); + } } /** @@ -875,7 +886,8 @@ protected void addConnection(TcpConnectionSupport connection) { * @return a list of open connection ids. */ private List removeClosedConnectionsAndReturnOpenConnectionIds() { - synchronized (this.connections) { + this.connectionsMonitor.lock(); + try { List openConnectionIds = new ArrayList<>(); Iterator> iterator = this.connections.entrySet().iterator(); while (iterator.hasNext()) { @@ -899,6 +911,9 @@ private List removeClosedConnectionsAndReturnOpenConnectionIds() { } return openConnectionIds; } + finally { + this.connectionsMonitor.unlock(); + } } /** @@ -959,7 +974,8 @@ public List getOpenConnectionIds() { public boolean closeConnection(String connectionId) { Assert.notNull(connectionId, "'connectionId' to close must not be null"); // closed connections are removed from #connections in #harvestClosedConnections() - synchronized (this.connections) { + this.connectionsMonitor.lock(); + try { boolean closed = false; TcpConnectionSupport connection = this.connections.remove(connectionId); if (connection != null) { @@ -975,6 +991,9 @@ public boolean closeConnection(String connectionId) { } return closed; } + finally { + this.connectionsMonitor.unlock(); + } } @Override diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java index fa7c2df42c4..45dad5b715f 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java @@ -37,6 +37,8 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import javax.net.ssl.SSLSession; @@ -72,14 +74,20 @@ public class TcpNioConnection extends TcpConnectionSupport { private static final byte[] EOF = new byte[0]; // EOF marker buffer + private final Lock lock = new ReentrantLock(); + private final SocketChannel socketChannel; + private final Lock socketChannelMonitor = new ReentrantLock(); + private final ChannelOutputStream channelOutputStream = new ChannelOutputStream(); private final ChannelInputStream channelInputStream = new ChannelInputStream(); private final AtomicInteger executionControl = new AtomicInteger(); + private final Lock executionControlMonitor = new ReentrantLock(); + private boolean usingDirectBuffers; private long pipeTimeout = DEFAULT_PIPE_TIMEOUT; @@ -154,7 +162,8 @@ public boolean isOpen() { @Override @SuppressWarnings("unchecked") public void send(Message message) { - synchronized (this.socketChannel) { + this.socketChannelMonitor.lock(); + try { try { if (this.bufferedOutputStream == null) { int writeBufferSize = this.socketChannel.socket().getSendBufferSize(); @@ -177,6 +186,9 @@ public void send(Message message) { logger.debug(getConnectionId() + " Message sent " + message); } } + finally { + this.socketChannelMonitor.unlock(); + } } @Override @@ -311,7 +323,8 @@ private boolean checkForMoreData() { // timing was such that we were the last assembler and // a new one wasn't run if (dataAvailable()) { - synchronized (this.executionControl) { + this.executionControlMonitor.lock(); + try { if (this.executionControl.incrementAndGet() <= 1) { // only continue if we don't already have another assembler running this.executionControl.set(1); @@ -322,6 +335,9 @@ private boolean checkForMoreData() { this.executionControl.decrementAndGet(); } } + finally { + this.executionControlMonitor.unlock(); + } } if (moreDataAvailable) { if (logger.isTraceEnabled()) { @@ -352,43 +368,50 @@ private boolean dataAvailable() { * @throws IOException an IO exception */ @Nullable - private synchronized Message convert() throws IOException { - if (logger.isTraceEnabled()) { - logger.trace(getConnectionId() + " checking data avail (convert): " + this.channelInputStream.available() + - " pending: " + (this.writingToPipe)); - } - if (this.channelInputStream.available() <= 0) { - try { - if (this.writingLatch.await(SIXTY, TimeUnit.SECONDS)) { - if (this.channelInputStream.available() <= 0) { - return null; + private Message convert() throws IOException { + this.lock.lock(); + try { + if (logger.isTraceEnabled()) { + logger.trace( + getConnectionId() + " checking data avail (convert): " + this.channelInputStream.available() + + " pending: " + (this.writingToPipe)); + } + if (this.channelInputStream.available() <= 0) { + try { + if (this.writingLatch.await(SIXTY, TimeUnit.SECONDS)) { + if (this.channelInputStream.available() <= 0) { + return null; + } + } + else { // should never happen + throw new IOException("Timed out waiting for IO"); } } - else { // should never happen - throw new IOException("Timed out waiting for IO"); + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted waiting for IO", e); } } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IOException("Interrupted waiting for IO", e); + try { + return getMapper().toMessage(this); } - } - try { - return getMapper().toMessage(this); - } - catch (Exception e) { - closeConnection(true); - if (e instanceof SocketTimeoutException) { // NOSONAR instanceof - if (logger.isDebugEnabled()) { - logger.debug("Closing socket after timeout " + getConnectionId()); + catch (Exception e) { + closeConnection(true); + if (e instanceof SocketTimeoutException) { // NOSONAR instanceof + if (logger.isDebugEnabled()) { + logger.debug("Closing socket after timeout " + getConnectionId()); + } } - } - else { - if (!(e instanceof SoftEndOfStreamException)) { // NOSONAR instanceof - throw e; + else { + if (!(e instanceof SoftEndOfStreamException)) { // NOSONAR instanceof + throw e; + } } + return null; } - return null; + } + finally { + this.lock.unlock(); } } @@ -468,7 +491,8 @@ protected void sendToPipe(ByteBuffer rawBufferToSend) throws IOException { } private void checkForAssembler() { - synchronized (this.executionControl) { + this.executionControlMonitor.lock(); + try { if (this.executionControl.incrementAndGet() <= 1) { // only execute run() if we don't already have one running this.executionControl.set(1); @@ -489,6 +513,9 @@ private void checkForAssembler() { this.executionControl.decrementAndGet(); } } + finally { + this.executionControlMonitor.unlock(); + } } /** @@ -605,6 +632,8 @@ class ChannelOutputStream extends OutputStream { private int soTimeout; + private final Lock innerLock = new ReentrantLock(); + @Override public void write(int b) throws IOException { byte[] bytes = new byte[1]; @@ -632,28 +661,35 @@ public void write(byte[] b) throws IOException { doWrite(buffer); } - protected synchronized void doWrite(ByteBuffer buffer) throws IOException { - if (logger.isDebugEnabled()) { - logger.debug(getConnectionId() + " writing " + buffer.remaining()); - } - TcpNioConnection.this.socketChannel.write(buffer); - int remaining = buffer.remaining(); - if (remaining == 0) { - return; - } - if (this.selector == null) { - this.selector = Selector.open(); - this.soTimeout = TcpNioConnection.this.socketChannel.socket().getSoTimeout(); - } - TcpNioConnection.this.socketChannel.register(this.selector, SelectionKey.OP_WRITE); - while (remaining > 0) { - int selectionCount = this.selector.select(this.soTimeout); - if (selectionCount == 0) { - throw new SocketTimeoutException("Timeout on write"); + protected void doWrite(ByteBuffer buffer) throws IOException { + this.innerLock.lock(); + try { + + if (logger.isDebugEnabled()) { + logger.debug(getConnectionId() + " writing " + buffer.remaining()); } - this.selector.selectedKeys().clear(); TcpNioConnection.this.socketChannel.write(buffer); - remaining = buffer.remaining(); + int remaining = buffer.remaining(); + if (remaining == 0) { + return; + } + if (this.selector == null) { + this.selector = Selector.open(); + this.soTimeout = TcpNioConnection.this.socketChannel.socket().getSoTimeout(); + } + TcpNioConnection.this.socketChannel.register(this.selector, SelectionKey.OP_WRITE); + while (remaining > 0) { + int selectionCount = this.selector.select(this.soTimeout); + if (selectionCount == 0) { + throw new SocketTimeoutException("Timeout on write"); + } + this.selector.selectedKeys().clear(); + TcpNioConnection.this.socketChannel.write(buffer); + remaining = buffer.remaining(); + } + } + finally { + this.innerLock.unlock(); } } @@ -680,6 +716,8 @@ class ChannelInputStream extends InputStream { private volatile boolean isClosed; + private final Lock innerLock = new ReentrantLock(); + @Override public int read(byte[] b, int off, int len) throws IOException { Assert.notNull(b, "byte[] cannot be null"); @@ -708,30 +746,36 @@ else if (len == 0) { } @Override - public synchronized int read() throws IOException { - if (this.isClosed && this.available.get() == 0) { - if (TcpNioConnection.this.timedOut) { - throw new SocketTimeoutException("Connection has timed out"); - } - return -1; - } - if (this.currentBuffer == null) { - this.currentBuffer = getNextBuffer(); - this.currentOffset = 0; - if (this.currentBuffer == null) { + public int read() throws IOException { + this.innerLock.lock(); + try { + if (this.isClosed && this.available.get() == 0) { if (TcpNioConnection.this.timedOut) { throw new SocketTimeoutException("Connection has timed out"); } return -1; } + if (this.currentBuffer == null) { + this.currentBuffer = getNextBuffer(); + this.currentOffset = 0; + if (this.currentBuffer == null) { + if (TcpNioConnection.this.timedOut) { + throw new SocketTimeoutException("Connection has timed out"); + } + return -1; + } + } + int bite; + bite = this.currentBuffer[this.currentOffset++] & 0xff; // NOSONAR + this.available.decrementAndGet(); + if (this.currentOffset >= this.currentBuffer.length) { + this.currentBuffer = null; + } + return bite; } - int bite; - bite = this.currentBuffer[this.currentOffset++] & 0xff; // NOSONAR - this.available.decrementAndGet(); - if (this.currentOffset >= this.currentBuffer.length) { - this.currentBuffer = null; + finally { + this.innerLock.unlock(); } - return bite; } @Nullable diff --git a/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java b/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java index 59de0f8f644..0116ce4b2d0 100644 --- a/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java +++ b/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java @@ -29,6 +29,8 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import jakarta.jms.Connection; import jakarta.jms.ConnectionFactory; @@ -90,7 +92,7 @@ public class JmsOutboundGateway extends AbstractReplyProducingMessageHandler */ public static final long DEFAULT_RECEIVE_TIMEOUT = 5000L; - private final Object initializationMonitor = new Object(); + private final Lock initializationMonitor = new ReentrantLock(); private final AtomicLong correlationId = new AtomicLong(); @@ -100,10 +102,12 @@ public class JmsOutboundGateway extends AbstractReplyProducingMessageHandler private final ConcurrentHashMap earlyOrLateReplies = new ConcurrentHashMap<>(); + private final Lock earlyOrLateRepliesMonitor = new ReentrantLock(); + private final Map>> futures = new ConcurrentHashMap<>(); - private final Object lifeCycleMonitor = new Object(); + private final Lock lifeCycleMonitor = new ReentrantLock(); private Destination requestDestination; @@ -512,7 +516,8 @@ private Destination resolveReplyDestination(String repDestinationName, Session s @Override protected void doInit() { - synchronized (this.initializationMonitor) { + this.initializationMonitor.lock(); + try { if (this.initialized) { return; } @@ -539,6 +544,9 @@ protected void doInit() { initializeReplyContainer(); this.initialized = true; } + finally { + this.initializationMonitor.unlock(); + } } private void initializeReplyContainer() { @@ -667,7 +675,8 @@ private void applyReplyContainerProperties(GatewayReplyListenerContainer contain @Override public void start() { - synchronized (this.lifeCycleMonitor) { + this.lifeCycleMonitor.lock(); + try { if (!this.active) { if (this.replyContainer != null) { TaskScheduler taskScheduler = getTaskScheduler(); @@ -689,11 +698,15 @@ public void start() { this.active = true; } } + finally { + this.lifeCycleMonitor.unlock(); + } } @Override public void stop() { - synchronized (this.lifeCycleMonitor) { + this.lifeCycleMonitor.lock(); + try { if (this.replyContainer != null) { this.replyContainer.shutdown(); this.wasStopped = true; @@ -708,6 +721,10 @@ public void stop() { } this.active = false; } + finally { + this.lifeCycleMonitor.unlock(); + } + } @Override @@ -727,7 +744,8 @@ protected Object handleRequestMessage(final Message requestMessage) { } else { if (this.idleReplyContainerTimeout > 0) { - synchronized (this.lifeCycleMonitor) { + this.lifeCycleMonitor.lock(); + try { this.lastSend = System.currentTimeMillis(); if (!this.replyContainer.isRunning()) { logger.debug(() -> getComponentName() + ": Starting reply container."); @@ -738,6 +756,9 @@ protected Object handleRequestMessage(final Message requestMessage) { Duration.ofMillis(this.idleReplyContainerTimeout / 2)); } } + finally { + this.lifeCycleMonitor.unlock(); + } } reply = sendAndReceiveWithContainer(requestMessage); } @@ -1121,13 +1142,17 @@ private jakarta.jms.Message doSendAndReceiveAsyncDefaultCorrelation(Destination /* * Check to see if the reply arrived before we obtained the correlationId */ - synchronized (this.earlyOrLateReplies) { + this.earlyOrLateRepliesMonitor.lock(); + try { TimedReply timedReply = this.earlyOrLateReplies.remove(correlation); if (timedReply != null) { logger.debug(() -> "Found early reply with correlationId " + correlationToLog); replyQueue.add(timedReply.getReply()); } } + finally { + this.earlyOrLateRepliesMonitor.unlock(); + } return obtainReplyFromContainer(correlation, replyQueue); } @@ -1298,13 +1323,17 @@ private void onMessageSync(jakarta.jms.Message message, String correlationId) { } throw new IllegalStateException("No sender waiting for reply"); } - synchronized (this.earlyOrLateReplies) { + this.earlyOrLateRepliesMonitor.lock(); + try { queue = this.replies.get(correlationId); if (queue == null) { logger.debug(() -> "Reply for correlationId " + correlationId + " received early or late"); this.earlyOrLateReplies.put(correlationId, new TimedReply(message)); } } + finally { + this.earlyOrLateRepliesMonitor.unlock(); + } } if (queue != null) { logger.debug(() -> "Received reply with correlationId " + correlationId); @@ -1466,7 +1495,8 @@ private class IdleContainerStopper implements Runnable { @Override public void run() { - synchronized (JmsOutboundGateway.this.lifeCycleMonitor) { + JmsOutboundGateway.this.lifeCycleMonitor.lock(); + try { if (System.currentTimeMillis() - JmsOutboundGateway.this.lastSend > JmsOutboundGateway.this.idleReplyContainerTimeout && JmsOutboundGateway.this.replies.size() == 0 && @@ -1478,6 +1508,9 @@ public void run() { JmsOutboundGateway.this.idleTask = null; } } + finally { + JmsOutboundGateway.this.lifeCycleMonitor.unlock(); + } } } diff --git a/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java b/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java index 7f621e0264a..ebae67a2a6e 100644 --- a/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java +++ b/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java @@ -30,6 +30,8 @@ import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -111,11 +113,13 @@ public class KafkaMessageSource extends AbstractMessageSource impl */ public static final String REMAINING_RECORDS = KafkaHeaders.PREFIX + "remainingRecords"; + private final Lock lock = new ReentrantLock(); + private final ConsumerFactory consumerFactory; private final KafkaAckCallbackFactory ackCallbackFactory; - private final Object consumerMonitor = new Object(); + private final Lock consumerMonitor = new ReentrantLock(); private final Map>> inflightRecords = new ConcurrentHashMap<>(); @@ -385,31 +389,61 @@ private boolean maxPollStringGtr1(Object maxPoll) { } @Override - public synchronized boolean isRunning() { - return this.running; + public boolean isRunning() { + this.lock.lock(); + try { + return this.running; + } + finally { + this.lock.unlock(); + } } @Override - public synchronized void start() { - this.running = true; - this.stopped = false; + public void start() { + this.lock.lock(); + try { + this.running = true; + this.stopped = false; + } + finally { + this.lock.unlock(); + } } @Override - public synchronized void stop() { - stopConsumer(); - this.running = false; - this.stopped = true; + public void stop() { + this.lock.lock(); + try { + stopConsumer(); + this.running = false; + this.stopped = true; + } + finally { + this.lock.unlock(); + } } @Override - public synchronized void pause() { - this.pausing = true; + public void pause() { + this.lock.lock(); + try { + this.pausing = true; + } + finally { + this.lock.unlock(); + } } @Override - public synchronized void resume() { - this.pausing = false; + public void resume() { + this.lock.lock(); + try { + this.pausing = false; + } + finally { + this.lock.unlock(); + } } @Override @@ -418,35 +452,43 @@ public boolean isPaused() { } @Override // NOSONAR - not so complex - protected synchronized Object doReceive() { - if (this.stopped) { - this.logger.debug("Message source is stopped; no records will be returned"); - return null; - } - if (this.consumer == null) { - createConsumer(); - this.running = true; - } - if (this.pausing && !this.paused && this.assignedPartitions.size() > 0) { - this.consumer.pause(this.assignedPartitions); - this.paused = true; - } - else if (this.paused && !this.pausing) { - this.consumer.resume(this.assignedPartitions); - this.paused = false; + protected Object doReceive() { + this.lock.lock(); + try { + + if (this.stopped) { + this.logger.debug("Message source is stopped; no records will be returned"); + return null; + } + if (this.consumer == null) { + createConsumer(); + this.running = true; + } + if (this.pausing && !this.paused && this.assignedPartitions.size() > 0) { + this.consumer.pause(this.assignedPartitions); + this.paused = true; + } + else if (this.paused && !this.pausing) { + this.consumer.resume(this.assignedPartitions); + this.paused = false; + } + if (this.paused && this.recordsIterator == null) { + this.logger.debug("Consumer is paused; no records will be returned"); + } + ConsumerRecord record = pollRecord(); + + return record != null + ? recordToMessage(record) + : null; } - if (this.paused && this.recordsIterator == null) { - this.logger.debug("Consumer is paused; no records will be returned"); + finally { + this.lock.unlock(); } - ConsumerRecord record = pollRecord(); - - return record != null - ? recordToMessage(record) - : null; } protected void createConsumer() { - synchronized (this.consumerMonitor) { + this.consumerMonitor.lock(); + try { this.consumer = this.consumerFactory.createConsumer(this.consumerProperties.getGroupId(), this.consumerProperties.getClientId(), null, this.consumerProperties.getKafkaConsumerProperties()); @@ -466,6 +508,9 @@ else if (partitions != null) { rebalanceCallback); } } + finally { + this.consumerMonitor.unlock(); + } } private void assignAndSeekPartitions(TopicPartitionOffset[] partitions) { @@ -522,7 +567,8 @@ private ConsumerRecord pollRecord() { return nextRecord(); } else { - synchronized (this.consumerMonitor) { + this.consumerMonitor.lock(); + try { try { ConsumerRecords records = this.consumer .poll(this.assignedPartitions.isEmpty() ? this.assignTimeout : this.pollTimeout); @@ -545,6 +591,9 @@ private ConsumerRecord pollRecord() { return null; } } + finally { + this.consumerMonitor.unlock(); + } } } @@ -590,18 +639,28 @@ private Object recordToMessage(ConsumerRecord record) { } @Override - public synchronized void destroy() { - stopConsumer(); + public void destroy() { + this.lock.lock(); + try { + stopConsumer(); + } + finally { + this.lock.unlock(); + } } private void stopConsumer() { - synchronized (this.consumerMonitor) { + this.consumerMonitor.lock(); + try { if (this.consumer != null) { this.consumer.close(this.closeTimeout); this.consumer = null; this.assignedPartitions.clear(); } } + finally { + this.consumerMonitor.unlock(); + } } private class IntegrationConsumerRebalanceListener implements ConsumerRebalanceListener { diff --git a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java index 7ef57952e6d..fa8706464bc 100644 --- a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java +++ b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java @@ -25,6 +25,8 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; import org.apache.commons.logging.Log; @@ -78,9 +80,11 @@ public abstract class AbstractStompSessionManager implements StompSessionManager protected final StompClientSupport stompClient; // NOSONAR final + private final Lock lock = new ReentrantLock(); + private final CompositeStompSessionHandler compositeStompSessionHandler = new CompositeStompSessionHandler(); - private final Object lifecycleMonitor = new Object(); + private final Lock lifecycleMonitor = new ReentrantLock(); private final AtomicInteger epoch = new AtomicInteger(); @@ -177,41 +181,47 @@ public int getPhase() { return this.phase; } - private synchronized void connect() { - if (this.connecting || this.connected) { - this.logger.debug("Aborting connect; another thread is connecting."); - return; - } - final int currentEpoch = this.epoch.get(); - this.connecting = true; - if (this.logger.isDebugEnabled()) { - this.logger.debug("Connecting " + this); - } + private void connect() { + this.lock.lock(); try { - this.stompSessionFuture = doConnect(this.compositeStompSessionHandler); - } - catch (Exception e) { - if (currentEpoch == this.epoch.get()) { - scheduleReconnect(e); + if (this.connecting || this.connected) { + this.logger.debug("Aborting connect; another thread is connecting."); + return; } - else { - this.logger.error("STOMP doConnect() error for " + this, e); + final int currentEpoch = this.epoch.get(); + this.connecting = true; + if (this.logger.isDebugEnabled()) { + this.logger.debug("Connecting " + this); } - return; - } - CountDownLatch connectLatch = addStompSessionCallback(currentEpoch); - - try { - if (!connectLatch.await(30, TimeUnit.SECONDS)) { // NOSONAR magic number - this.logger.error("No response to connection attempt"); + try { + this.stompSessionFuture = doConnect(this.compositeStompSessionHandler); + } + catch (Exception e) { if (currentEpoch == this.epoch.get()) { - scheduleReconnect(null); + scheduleReconnect(e); + } + else { + this.logger.error("STOMP doConnect() error for " + this, e); + } + return; + } + CountDownLatch connectLatch = addStompSessionCallback(currentEpoch); + + try { + if (!connectLatch.await(30, TimeUnit.SECONDS)) { // NOSONAR magic number + this.logger.error("No response to connection attempt"); + if (currentEpoch == this.epoch.get()) { + scheduleReconnect(null); + } } } + catch (InterruptedException e1) { + this.logger.error("Interrupted while waiting for connection attempt"); + Thread.currentThread().interrupt(); + } } - catch (InterruptedException e1) { - this.logger.error("Interrupted while waiting for connection attempt"); - Thread.currentThread().interrupt(); + finally { + this.lock.unlock(); } } @@ -294,7 +304,8 @@ public void accept(StompSession session, Throwable throwable) { @Override public void start() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.lock(); + try { if (!isRunning()) { if (this.logger.isInfoEnabled()) { this.logger.info("Starting " + this); @@ -303,11 +314,15 @@ public void start() { this.running = true; } } + finally { + this.lifecycleMonitor.unlock(); + } } @Override public void stop() { - synchronized (this.lifecycleMonitor) { + this.lifecycleMonitor.lock(); + try { if (isRunning()) { this.running = false; if (this.logger.isInfoEnabled()) { @@ -316,6 +331,9 @@ public void stop() { destroy(); } } + finally { + this.lifecycleMonitor.unlock(); + } } @Override @@ -355,18 +373,24 @@ private class CompositeStompSessionHandler extends StompSessionHandlerAdapter { private final List delegates = Collections.synchronizedList(new ArrayList<>()); + private final Lock delegatesMonitor = new ReentrantLock(); + private volatile StompSession session; CompositeStompSessionHandler() { } void addHandler(StompSessionHandler delegate) { - synchronized (this.delegates) { + this.delegatesMonitor.lock(); + try { if (this.session != null) { delegate.afterConnected(this.session, getConnectHeaders()); } this.delegates.add(delegate); } + finally { + this.delegatesMonitor.lock(); + } } void removeHandler(StompSessionHandler delegate) { @@ -375,23 +399,31 @@ void removeHandler(StompSessionHandler delegate) { @Override public void afterConnected(StompSession session, StompHeaders connectedHeaders) { - synchronized (this.delegates) { + this.delegatesMonitor.lock(); + try { this.session = session; for (StompSessionHandler delegate : this.delegates) { delegate.afterConnected(session, connectedHeaders); } } + finally { + this.delegatesMonitor.unlock(); + } } @Override public void handleException(StompSession session, @Nullable StompCommand command, StompHeaders headers, byte[] payload, Throwable exception) { - synchronized (this.delegates) { + this.delegatesMonitor.lock(); + try { for (StompSessionHandler delegate : this.delegates) { delegate.handleException(session, command, headers, payload, exception); } } + finally { + this.delegatesMonitor.unlock(); + } } @Override @@ -400,20 +432,28 @@ public void handleTransportError(StompSession session, Throwable exception) { exception); this.session = null; scheduleReconnect(exception); - synchronized (this.delegates) { + this.delegatesMonitor.lock(); + try { for (StompSessionHandler delegate : this.delegates) { delegate.handleTransportError(session, exception); } } + finally { + this.delegatesMonitor.unlock(); + } } @Override public void handleFrame(StompHeaders headers, Object payload) { - synchronized (this.delegates) { + this.delegatesMonitor.lock(); + try { for (StompSessionHandler delegate : this.delegates) { delegate.handleFrame(headers, payload); } } + finally { + this.delegatesMonitor.unlock(); + } } } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java index 76bd1ac3224..4b3bd114303 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java @@ -16,6 +16,9 @@ package org.springframework.integration.xml.result; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; @@ -35,6 +38,10 @@ public class DomResultFactory implements ResultFactory { private final DocumentBuilderFactory documentBuilderFactory; + private final Lock documentBuilderFactoryMonitor = new ReentrantLock(); + + private final Lock lock = new ReentrantLock(); + public DomResultFactory() { this(DocumentBuilderFactoryUtils.newInstance()); @@ -48,7 +55,8 @@ public DomResultFactory(DocumentBuilderFactory documentBuilderFactory) { @Override - public synchronized Result createResult(Object payload) { + public Result createResult(Object payload) { + this.lock.lock(); try { return new DOMResult(getNewDocumentBuilder().newDocument()); } @@ -56,12 +64,19 @@ public synchronized Result createResult(Object payload) { throw new MessagingException("failed to create Result for payload type [" + payload.getClass().getName() + "]", e); } + finally { + this.lock.unlock(); + } } protected DocumentBuilder getNewDocumentBuilder() throws ParserConfigurationException { - synchronized (this.documentBuilderFactory) { + this.documentBuilderFactoryMonitor.lock(); + try { return this.documentBuilderFactory.newDocumentBuilder(); } + finally { + this.documentBuilderFactoryMonitor.unlock(); + } } } diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java index 3d615024fa3..f5016bc7ee6 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java @@ -22,6 +22,8 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; import javax.xml.parsers.DocumentBuilder; @@ -72,7 +74,9 @@ public class XPathMessageSplitter extends AbstractMessageSplitter { private final TransformerFactory transformerFactory; - private final Object documentBuilderFactoryMonitor = new Object(); + private final Lock documentBuilderFactoryMonitor = new ReentrantLock(); + + private final Lock transformerFactoryMonitor = new ReentrantLock(); private final XPathExpression xpathExpression; @@ -249,9 +253,13 @@ protected int obtainSizeIfPossible(Iterator iterator) { private Object splitDocument(Document document) throws ParserConfigurationException, TransformerException { Object nodes = splitNode(document); final Transformer transformer; - synchronized (this.transformerFactory) { + this.transformerFactoryMonitor.lock(); + try { transformer = this.transformerFactory.newTransformer(); } + finally { + this.transformerFactoryMonitor.unlock(); + } if (this.outputProperties != null) { transformer.setOutputProperties(this.outputProperties); } @@ -317,9 +325,13 @@ private Document convertNodeToDocument(DocumentBuilder documentBuilder, Node nod } private DocumentBuilder getNewDocumentBuilder() throws ParserConfigurationException { - synchronized (this.documentBuilderFactoryMonitor) { + this.documentBuilderFactoryMonitor.lock(); + try { return this.documentBuilderFactory.newDocumentBuilder(); } + finally { + this.documentBuilderFactoryMonitor.unlock(); + } } private final class NodeListIterator implements Iterator { From a0ed624c996cc3ff9bd61d84a0f2f161e2f71126 Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Fri, 16 Jun 2023 18:01:30 +0200 Subject: [PATCH 05/10] javadoc + year --- .../expression/ReloadableResourceBundleExpressionSource.java | 1 + .../integration/ip/tcp/connection/TcpNioConnection.java | 1 + .../springframework/integration/jms/JmsOutboundGateway.java | 3 ++- .../integration/kafka/inbound/KafkaMessageSource.java | 3 ++- .../integration/stomp/AbstractStompSessionManager.java | 3 ++- .../integration/xml/result/DomResultFactory.java | 3 ++- .../integration/xml/splitter/XPathMessageSplitter.java | 3 ++- 7 files changed, 12 insertions(+), 5 deletions(-) diff --git a/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java b/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java index d065bf1033f..37d3d7fb339 100644 --- a/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java +++ b/spring-integration-core/src/main/java/org/springframework/integration/expression/ReloadableResourceBundleExpressionSource.java @@ -55,6 +55,7 @@ * @author Mark Fisher * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov * * @since 2.0 * diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java index 45dad5b715f..21afc3979d1 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/tcp/connection/TcpNioConnection.java @@ -58,6 +58,7 @@ * @author John Anderson * @author Artem Bilan * @author David Herschler Shvo + * @author Christian Tzolov * * @since 2.0 * diff --git a/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java b/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java index 0116ce4b2d0..8a07a7860bf 100644 --- a/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java +++ b/spring-integration-jms/src/main/java/org/springframework/integration/jms/JmsOutboundGateway.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2022 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -83,6 +83,7 @@ * @author Oleg Zhurakousky * @author Gary Russell * @author Artem Bilan + * @author Christian Tzolov */ public class JmsOutboundGateway extends AbstractReplyProducingMessageHandler implements ManageableLifecycle, MessageListener { diff --git a/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java b/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java index ebae67a2a6e..96d1e18180e 100644 --- a/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java +++ b/spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java @@ -1,5 +1,5 @@ /* - * Copyright 2018-2022 the original author or authors. + * Copyright 2018-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -97,6 +97,7 @@ * @author Mark Norkin * @author Artem Bilan * @author Anshul Mehra + * @author Christian Tzolov * * @since 5.4 * diff --git a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java index fa8706464bc..fa7e9c8e3c6 100644 --- a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java +++ b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2022 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -68,6 +68,7 @@ * * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov * * @since 4.2 */ diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java index 4b3bd114303..4ee51dab409 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/result/DomResultFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2019 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -33,6 +33,7 @@ * @author Jonas Partner * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov */ public class DomResultFactory implements ResultFactory { diff --git a/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java b/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java index f5016bc7ee6..13eb3653882 100644 --- a/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java +++ b/spring-integration-xml/src/main/java/org/springframework/integration/xml/splitter/XPathMessageSplitter.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2021 the original author or authors. + * Copyright 2002-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -69,6 +69,7 @@ * @author Mark Fisher * @author Artem Bilan * @author Gary Russell + * @author Christian Tzolov */ public class XPathMessageSplitter extends AbstractMessageSplitter { From f7c59a81794ee0bf66e0b604d035fbb747902f0e Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Fri, 16 Jun 2023 19:39:29 +0200 Subject: [PATCH 06/10] addres first batch of review suggestions --- .../core/TimeBasedUUIDGenerator.java | 14 +++------ .../file/locking/FileChannelCache.java | 2 +- .../udp/MulticastReceivingChannelAdapter.java | 2 -- .../udp/MulticastSendingMessageHandler.java | 2 -- .../udp/UnicastReceivingChannelAdapter.java | 2 +- .../ip/udp/UnicastSendingMessageHandler.java | 2 +- .../tcp/connection/HelloWorldInterceptor.java | 14 ++------- .../integration/jdbc/LockInterceptor.java | 16 ++-------- .../lock/JdbcLockRegistryDelegateTests.java | 10 +++---- .../integration/jpa/test/Consumer.java | 30 ++++--------------- .../mqtt/core/AbstractMqttClientManager.java | 2 +- .../mqtt/core/Mqttv3ClientManager.java | 2 -- .../mqtt/core/Mqttv5ClientManager.java | 2 -- ...stractMqttMessageDrivenChannelAdapter.java | 26 ++++++++-------- .../MqttPahoMessageDrivenChannelAdapter.java | 14 ++++----- ...Mqttv5PahoMessageDrivenChannelAdapter.java | 18 +++++------ .../outbound/AbstractMqttMessageHandler.java | 4 +++ .../mqtt/outbound/MqttPahoMessageHandler.java | 2 -- .../websocket/ClientWebSocketContainer.java | 2 -- .../IntegrationWebSocketContainer.java | 4 +++ .../websocket/ServerWebSocketContainer.java | 2 -- .../config/LeaderInitiatorFactoryBean.java | 10 +------ 22 files changed, 60 insertions(+), 122 deletions(-) diff --git a/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java b/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java index ad0fdf6c5ff..f4f41d93433 100644 --- a/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java +++ b/spring-integration-core/src/test/java/org/springframework/integration/core/TimeBasedUUIDGenerator.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2023 the original author or authors. + * Copyright 2002-2022 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,20 +19,17 @@ import java.net.InetAddress; import java.net.NetworkInterface; import java.util.UUID; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.logging.Logger; /** * @author Oleg Zhurakousky * @author Artem Bilan - * @author Christian Tzolov * */ class TimeBasedUUIDGenerator { - static final Lock lock = new ReentrantLock(); + static final Object lock = new Object(); private static final Logger logger = Logger.getLogger(TimeBasedUUIDGenerator.class.getName()); @@ -59,8 +56,7 @@ static UUID generateId() { static UUID generateIdFromTimestamp(long currentTimeMillis) { long time; - lock.lock(); - try { + synchronized (lock) { if (currentTimeMillis > lastTime) { lastTime = currentTimeMillis; clockSequence = 0; @@ -69,9 +65,7 @@ static UUID generateIdFromTimestamp(long currentTimeMillis) { ++clockSequence; } } - finally { - lock.unlock(); - } + time = currentTimeMillis; diff --git a/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java b/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java index b52806500f1..adbc63110d9 100644 --- a/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java +++ b/spring-integration-file/src/main/java/org/springframework/integration/file/locking/FileChannelCache.java @@ -76,7 +76,7 @@ public static FileLock tryLockFor(File fileToLock) throws IOException { FileLock lock = null; if (channel != null) { try { - lock = channel.lock(); + lock = channel.tryLock(); } catch (OverlappingFileLockException e) { // File is already locked in this thread or virtual machine diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java index 0af8c6d3ecd..e5c586413b4 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java @@ -42,8 +42,6 @@ public class MulticastReceivingChannelAdapter extends UnicastReceivingChannelAda private final String group; - private final Lock lock = new ReentrantLock(); - /** * Constructs a MulticastReceivingChannelAdapter that listens for packets on the * specified multichannel address (group) and port. diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java index 1b5b0a19536..9bf71865f8f 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java @@ -51,8 +51,6 @@ public class MulticastSendingMessageHandler extends UnicastSendingMessageHandler private volatile MulticastSocket multicastSocket; - private final Lock lock = new ReentrantLock(); - /** * Constructs a MulticastSendingMessageHandler to send data to the multicast address/port. * @param address The multicast address. diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java index 4569390329f..db8b380c3e1 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastReceivingChannelAdapter.java @@ -56,7 +56,7 @@ public class UnicastReceivingChannelAdapter extends AbstractInternetProtocolRece private final DatagramPacketMessageMapper mapper = new DatagramPacketMessageMapper(); - private final Lock lock = new ReentrantLock(); + protected final Lock lock = new ReentrantLock(); private DatagramSocket socket; diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java index 77a4f757514..ba638af6ae0 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/UnicastSendingMessageHandler.java @@ -69,7 +69,7 @@ public class UnicastSendingMessageHandler extends private static final int DEFAULT_ACK_TIMEOUT = 5000; - private final Lock lock = new ReentrantLock(); + protected final Lock lock = new ReentrantLock(); private final DatagramPacketMessageMapper mapper = new DatagramPacketMessageMapper(); diff --git a/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java b/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java index 3b16447cf2d..a85d9f890e8 100644 --- a/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java +++ b/spring-integration-ip/src/test/java/org/springframework/integration/ip/tcp/connection/HelloWorldInterceptor.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2023 the original author or authors. + * Copyright 2002-2021 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,8 +18,6 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.ApplicationEventPublisher; import org.springframework.integration.support.MessageBuilder; @@ -28,15 +26,11 @@ /** * @author Gary Russell - * @author Christian Tzolov - * * @since 2.0 * */ public class HelloWorldInterceptor extends TcpConnectionInterceptorSupport { - private final Lock lock = new ReentrantLock(); - private volatile boolean negotiated; private final Semaphore negotiationSemaphore = new Semaphore(0); @@ -63,8 +57,7 @@ public HelloWorldInterceptor(String hello, String world, ApplicationEventPublish @Override public boolean onMessage(Message message) { if (!this.negotiated) { - this.lock.lock(); - try { + synchronized (this) { if (!this.negotiated) { Object payload = message.getPayload(); logger.debug(this.toString() + " received " + payload); @@ -98,9 +91,6 @@ public boolean onMessage(Message message) { } } } - finally { - this.lock.unlock(); - } } try { return super.onMessage(message); diff --git a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java index e181b7a6742..b5c29573dbc 100644 --- a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java +++ b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/LockInterceptor.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2023 the original author or authors. + * Copyright 2002-2019 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,7 +16,6 @@ package org.springframework.integration.jdbc; -import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import org.aopalliance.intercept.MethodInterceptor; @@ -24,23 +23,14 @@ /** * @author Dave Syer - * @author Christian Tzolov * */ public class LockInterceptor extends ReentrantLock implements MethodInterceptor { private static final long serialVersionUID = 1L; - private final Lock lock = new ReentrantLock(); - - public Object invoke(MethodInvocation invocation) throws Throwable { - this.lock.lock(); - try { - return invocation.proceed(); - } - finally { - this.lock.unlock(); - } + public synchronized Object invoke(MethodInvocation invocation) throws Throwable { + return invocation.proceed(); } } diff --git a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java index 20abbd88de6..2ff6a720d8c 100644 --- a/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java +++ b/spring-integration-jdbc/src/test/java/org/springframework/integration/jdbc/lock/JdbcLockRegistryDelegateTests.java @@ -63,7 +63,7 @@ public void testLessAmountOfUnlockThanLock() { final Lock lock = registry.obtain("foo"); for (int i = 0; i < lockCount; i++) { - lock.lock(); + lock.tryLock(); } for (int i = 0; i < unlockCount; i++) { lock.unlock(); @@ -79,7 +79,7 @@ public void testSameAmountOfUnlockThanLock() { final Lock lock = registry.obtain("foo"); for (int i = 0; i < lockCount; i++) { - lock.lock(); + lock.tryLock(); } for (int i = 0; i < lockCount; i++) { lock.unlock(); @@ -91,7 +91,7 @@ public void testSameAmountOfUnlockThanLock() { @Test public void testTransientDataAccessException() { final Lock lock = registry.obtain("foo"); - lock.lock(); + lock.tryLock(); final AtomicBoolean shouldThrow = new AtomicBoolean(true); doAnswer(invocation -> { @@ -109,7 +109,7 @@ public void testTransientDataAccessException() { @Test public void testTransactionTimedOutException() { final Lock lock = registry.obtain("foo"); - lock.lock(); + lock.tryLock(); final AtomicBoolean shouldThrow = new AtomicBoolean(true); doAnswer(invocation -> { @@ -127,7 +127,7 @@ public void testTransactionTimedOutException() { @Test public void testTransactionSystemException() { final Lock lock = registry.obtain("foo"); - lock.lock(); + lock.tryLock(); final AtomicBoolean shouldThrow = new AtomicBoolean(true); doAnswer(invocation -> { diff --git a/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java b/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java index 3238bcf2ca9..f8e14351106 100644 --- a/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java +++ b/spring-integration-jpa/src/test/java/org/springframework/integration/jpa/test/Consumer.java @@ -1,5 +1,5 @@ /* - * Copyright 2002-2023 the original author or authors. + * Copyright 2002-2019 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,8 +20,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -31,8 +29,6 @@ /** * * @author Gunnar Hillert - * @author Christian Tzolov - * * @since 2.2 * */ @@ -42,27 +38,13 @@ public final class Consumer { private static final BlockingQueue>> MESSAGES = new LinkedBlockingQueue>>(); - private final Lock lock = new ReentrantLock(); - - public void receive(Message> message) { - this.lock.lock(); - try { - logger.info("Service Activator received Message: " + message); - MESSAGES.add(message); - } - finally { - this.lock.unlock(); - } + public synchronized void receive(Message> message) { + logger.info("Service Activator received Message: " + message); + MESSAGES.add(message); } - public Message> poll(long timeoutInMillis) throws InterruptedException { - this.lock.lock(); - try { - return MESSAGES.poll(timeoutInMillis, TimeUnit.MILLISECONDS); - } - finally { - this.lock.unlock(); - } + public synchronized Message> poll(long timeoutInMillis) throws InterruptedException { + return MESSAGES.poll(timeoutInMillis, TimeUnit.MILLISECONDS); } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java index 61446eebc62..0a5e2a9012a 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/AbstractMqttClientManager.java @@ -50,7 +50,7 @@ public abstract class AbstractMqttClientManager implements ClientManager connectCallbacks = Collections.synchronizedSet(new HashSet<>()); diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java index be77b94cf7b..b3b999880a1 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java @@ -47,8 +47,6 @@ public class Mqttv3ClientManager extends AbstractMqttClientManager implements MqttCallbackExtended { - private final Lock lock = new ReentrantLock(); - private final MqttConnectOptions connectionOptions; private MqttClientPersistence persistence; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java index 2e3cad6e61d..6298a803b0b 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java @@ -49,8 +49,6 @@ public class Mqttv5ClientManager extends AbstractMqttClientManager implements MqttCallback { - private final Lock lock = new ReentrantLock(); - private final MqttConnectionOptions connectionOptions; private MqttClientPersistence persistence; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java index 6712b00687d..243f1e2449b 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java @@ -58,7 +58,7 @@ public abstract class AbstractMqttMessageDrivenChannelAdapter extends MessageProducerSupport implements ApplicationEventPublisherAware, ClientManager.ConnectCallback { - protected final Lock topicLock = new ReentrantLock(); // NOSONAR + protected final Lock lock = new ReentrantLock(); // NOSONAR private final String url; @@ -137,7 +137,7 @@ public void setQos(int... qos) { @ManagedAttribute public int[] getQos() { - this.topicLock.lock(); + this.lock.lock(); try { int[] topicQos = new int[this.topics.size()]; int n = 0; @@ -147,7 +147,7 @@ public int[] getQos() { return topicQos; } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @@ -167,12 +167,12 @@ protected MqttMessageConverter getConverter() { @ManagedAttribute public String[] getTopic() { - this.topicLock.lock(); + this.lock.lock(); try { return this.topics.keySet().toArray(new String[0]); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @@ -256,7 +256,7 @@ protected long getCompletionTimeout() { */ @ManagedOperation public void addTopic(String topic, int qos) { - this.topicLock.lock(); + this.lock.lock(); try { if (this.topics.containsKey(topic)) { throw new MessagingException("Topic '" + topic + "' is already subscribed."); @@ -265,7 +265,7 @@ public void addTopic(String topic, int qos) { logger.debug(LogMessage.format("Added '%s' to subscriptions.", topic)); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @@ -278,14 +278,14 @@ public void addTopic(String topic, int qos) { @ManagedOperation public void addTopic(String... topic) { Assert.notNull(topic, "'topic' cannot be null"); - this.topicLock.lock(); + this.lock.lock(); try { for (String t : topic) { addTopic(t, 1); } } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @@ -301,7 +301,7 @@ public void addTopics(String[] topic, int[] qos) { Assert.notNull(topic, "'topic' cannot be null."); Assert.noNullElements(topic, "'topic' cannot contain any null elements."); Assert.isTrue(topic.length == qos.length, "topic and qos arrays must the be the same length."); - this.topicLock.lock(); + this.lock.lock(); try { for (String newTopic : topic) { if (this.topics.containsKey(newTopic)) { @@ -313,7 +313,7 @@ public void addTopics(String[] topic, int[] qos) { } } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @@ -325,7 +325,7 @@ public void addTopics(String[] topic, int[] qos) { */ @ManagedOperation public void removeTopic(String... topic) { - this.topicLock.lock(); + this.lock.lock(); try { for (String name : topic) { if (this.topics.remove(name) != null) { @@ -334,7 +334,7 @@ public void removeTopic(String... topic) { } } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java index a960d2f4efb..ff030838583 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java @@ -67,8 +67,6 @@ public class MqttPahoMessageDrivenChannelAdapter extends AbstractMqttMessageDrivenChannelAdapter implements MqttCallbackExtended, MqttPahoComponent { - private final Lock lock = new ReentrantLock(); - private final MqttPahoClientFactory clientFactory; private volatile IMqttAsyncClient client; @@ -254,7 +252,7 @@ public void destroy() { @Override public void addTopic(String topic, int qos) { - this.topicLock.lock(); + this.lock.lock(); try { super.addTopic(topic, qos); if (this.client != null && this.client.isConnected()) { @@ -267,13 +265,13 @@ public void addTopic(String topic, int qos) { throw new MessagingException("Failed to subscribe to topic " + topic, e); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @Override public void removeTopic(String... topic) { - this.topicLock.lock(); + this.lock.lock(); try { if (this.client != null && this.client.isConnected()) { this.client.unsubscribe(topic).waitForCompletion(getCompletionTimeout()); @@ -284,12 +282,12 @@ public void removeTopic(String... topic) { throw new MessagingException("Failed to unsubscribe from topic(s) " + Arrays.toString(topic), e); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } private void subscribe() { - this.topicLock.lock(); + this.lock.lock(); String[] topics = getTopic(); ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); try { @@ -316,7 +314,7 @@ private void subscribe() { logger.error(ex, () -> "Error subscribing to " + Arrays.toString(topics)); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } if (this.client.isConnected()) { String message = "Connected and subscribed to " + Arrays.toString(topics); diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java index 988b094eeb0..6744f6d9180 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java @@ -84,8 +84,6 @@ public class Mqttv5PahoMessageDrivenChannelAdapter extends AbstractMqttMessageDrivenChannelAdapter implements MqttCallback, MqttComponent { - private final Lock lock = new ReentrantLock(); - private final MqttConnectionOptions connectionOptions; private IMqttAsyncClient mqttClient; @@ -234,7 +232,7 @@ private void connect() throws MqttException { @Override protected void doStop() { - this.topicLock.lock(); + this.lock.lock(); this.readyToSubscribeOnStart = false; String[] topics = getTopic(); try { @@ -254,7 +252,7 @@ protected void doStop() { logger.error(ex, () -> "Error unsubscribing from " + Arrays.toString(topics)); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @@ -273,7 +271,7 @@ public void destroy() { @Override public void addTopic(String topic, int qos) { - this.topicLock.lock(); + this.lock.lock(); try { super.addTopic(topic, qos); if (this.mqttClient != null && this.mqttClient.isConnected()) { @@ -285,13 +283,13 @@ public void addTopic(String topic, int qos) { throw new MessagingException("Failed to subscribe to topic " + topic, ex); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @Override public void removeTopic(String... topic) { - this.topicLock.lock(); + this.lock.lock(); try { if (this.mqttClient != null && this.mqttClient.isConnected()) { this.mqttClient.unsubscribe(topic).waitForCompletion(getCompletionTimeout()); @@ -302,7 +300,7 @@ public void removeTopic(String... topic) { throw new MessagingException("Failed to unsubscribe from topic(s) " + Arrays.toString(topic), ex); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } @@ -395,7 +393,7 @@ private void subscribe() { String[] topics = getTopic(); ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); - this.topicLock.lock(); + this.lock.lock(); try { if (topics.length == 0) { return; @@ -424,7 +422,7 @@ private void subscribe() { logger.error(ex, () -> "Error subscribing to " + Arrays.toString(topics)); } finally { - this.topicLock.unlock(); + this.lock.unlock(); } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/AbstractMqttMessageHandler.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/AbstractMqttMessageHandler.java index 387db0497fe..a17f7f63a5b 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/AbstractMqttMessageHandler.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/AbstractMqttMessageHandler.java @@ -17,6 +17,8 @@ package org.springframework.integration.mqtt.outbound; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.springframework.beans.factory.BeanFactoryAware; import org.springframework.context.ApplicationEventPublisher; @@ -63,6 +65,8 @@ public abstract class AbstractMqttMessageHandler extends AbstractMessageHa private static final MessageProcessor DEFAULT_TOPIC_PROCESSOR = (message) -> message.getHeaders().get(MqttHeaders.TOPIC, String.class); + protected final Lock lock = new ReentrantLock(); + private final AtomicBoolean running = new AtomicBoolean(); private final String url; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java index b5be9fbdbb8..5b35fb7e36b 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java @@ -61,8 +61,6 @@ public class MqttPahoMessageHandler extends AbstractMqttMessageHandler implements MqttCallback, MqttPahoComponent { - private final Lock lock = new ReentrantLock(); - private final MqttPahoClientFactory clientFactory; private boolean async; diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java index 11403648f04..ced6fe3522e 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java @@ -56,8 +56,6 @@ public final class ClientWebSocketContainer extends IntegrationWebSocketContaine private static final int DEFAULT_CONNECTION_TIMEOUT = 10; - private final Lock lock = new ReentrantLock(); - private final WebSocketHttpHeaders headers = new WebSocketHttpHeaders(); private final IntegrationWebSocketConnectionManager connectionManager; diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/IntegrationWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/IntegrationWebSocketContainer.java index fab87e2fd2e..dfe8d67e9c9 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/IntegrationWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/IntegrationWebSocketContainer.java @@ -21,6 +21,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -67,6 +69,8 @@ public abstract class IntegrationWebSocketContainer implements DisposableBean { protected final Log logger = LogFactory.getLog(getClass()); // NOSONAR + protected final Lock lock = new ReentrantLock(); + private WebSocketHandler webSocketHandler = new IntegrationWebSocketHandler(); protected final Map sessions = new ConcurrentHashMap<>(); // NOSONAR diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java index ee5d7b9614d..8f862d9374e 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java @@ -56,8 +56,6 @@ public class ServerWebSocketContainer extends IntegrationWebSocketContainer implements WebSocketConfigurer, SmartLifecycle { - private final Lock lock = new ReentrantLock(); - private final String[] paths; private HandshakeHandler handshakeHandler; diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java index a370c7c0b51..712c6e68eb8 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java @@ -46,8 +46,6 @@ public class LeaderInitiatorFactoryBean implements FactoryBean, SmartLifecycle, InitializingBean, ApplicationEventPublisherAware { - private final Lock lock = new ReentrantLock(); - private CuratorFramework client; private Candidate candidate; @@ -183,13 +181,7 @@ else if (this.applicationEventPublisher != null) { @Override public LeaderInitiator getObject() { - this.lock.lock(); - try { - return this.leaderInitiator; - } - finally { - this.lock.unlock(); - } + return this.leaderInitiator; } @Override From 03568a3515d4910636da78aa580d156c3566c075 Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Fri, 16 Jun 2023 19:56:44 +0200 Subject: [PATCH 07/10] fix checkstyle issues --- .../udp/MulticastReceivingChannelAdapter.java | 2 - .../udp/MulticastSendingMessageHandler.java | 2 - ...PostgresChannelMessageTableSubscriber.java | 123 +++++++++--------- .../mqtt/core/Mqttv3ClientManager.java | 3 - .../mqtt/core/Mqttv5ClientManager.java | 3 - .../MqttPahoMessageDrivenChannelAdapter.java | 2 - ...Mqttv5PahoMessageDrivenChannelAdapter.java | 2 - .../mqtt/outbound/MqttPahoMessageHandler.java | 3 - .../websocket/ClientWebSocketContainer.java | 2 - .../websocket/ServerWebSocketContainer.java | 2 - .../config/LeaderInitiatorFactoryBean.java | 2 - .../zookeeper/lock/ZookeeperLockRegistry.java | 3 +- 12 files changed, 66 insertions(+), 83 deletions(-) diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java index e5c586413b4..0dbaa0b064e 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastReceivingChannelAdapter.java @@ -22,8 +22,6 @@ import java.net.InetSocketAddress; import java.net.MulticastSocket; import java.net.NetworkInterface; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.springframework.messaging.MessagingException; diff --git a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java index 9bf71865f8f..b6b57190fc1 100644 --- a/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java +++ b/spring-integration-ip/src/main/java/org/springframework/integration/ip/udp/MulticastSendingMessageHandler.java @@ -23,8 +23,6 @@ import java.net.MulticastSocket; import java.net.NetworkInterface; import java.net.URISyntaxException; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.springframework.expression.Expression; import org.springframework.messaging.Message; diff --git a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java index fceb1381e3a..d0a5cd82902 100644 --- a/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java +++ b/spring-integration-jdbc/src/main/java/org/springframework/integration/jdbc/channel/PostgresChannelMessageTableSubscriber.java @@ -171,65 +171,7 @@ public void start() { CountDownLatch startingLatch = new CountDownLatch(1); this.future = this.taskExecutor.submit(() -> { - try { - while (isActive()) { - try { - PgConnection conn = this.connectionSupplier.get(); - try (Statement stmt = conn.createStatement()) { - stmt.execute("LISTEN " + this.tablePrefix.toLowerCase() + "channel_message_notify"); - } - catch (Exception ex) { - try { - conn.close(); - } - catch (Exception suppressed) { - ex.addSuppressed(suppressed); - } - throw ex; - } - this.subscriptionsMap.values() - .forEach(subscriptions -> subscriptions.forEach(Subscription::notifyUpdate)); - try { - this.connection = conn; - while (isActive()) { - startingLatch.countDown(); - - PGNotification[] notifications = conn.getNotifications(0); - // Unfortunately, there is no good way of interrupting a notification - // poll but by closing its connection. - if (!isActive()) { - return; - } - if (notifications != null) { - for (PGNotification notification : notifications) { - String parameter = notification.getParameter(); - Set subscriptions = this.subscriptionsMap.get(parameter); - if (subscriptions == null) { - continue; - } - for (Subscription subscription : subscriptions) { - subscription.notifyUpdate(); - } - } - } - } - } - finally { - conn.close(); - } - } - catch (Exception e) { - // The getNotifications method does not throw a meaningful message on interruption. - // Therefore, we do not log an error, unless it occurred while active. - if (isActive()) { - LOGGER.error(e, "Failed to poll notifications from Postgres database"); - } - } - } - } - finally { - this.latch.countDown(); - } + doStart(startingLatch); }); try { @@ -247,6 +189,69 @@ public void start() { } } + private void doStart(CountDownLatch startingLatch) { + try { + while (isActive()) { + try { + PgConnection conn = this.connectionSupplier.get(); + try (Statement stmt = conn.createStatement()) { + stmt.execute("LISTEN " + this.tablePrefix.toLowerCase() + "channel_message_notify"); + } + catch (Exception ex) { + try { + conn.close(); + } + catch (Exception suppressed) { + ex.addSuppressed(suppressed); + } + throw ex; + } + this.subscriptionsMap.values() + .forEach(subscriptions -> subscriptions.forEach(Subscription::notifyUpdate)); + try { + this.connection = conn; + while (isActive()) { + startingLatch.countDown(); + + PGNotification[] notifications = conn.getNotifications(0); + // Unfortunately, there is no good way of interrupting a notification + // poll but by closing its connection. + if (!isActive()) { + return; + } + if (notifications != null) { + for (PGNotification notification : notifications) { + String parameter = notification.getParameter(); + Set subscriptions = this.subscriptionsMap.get(parameter); + if (subscriptions == null) { + continue; + } + for (Subscription subscription : subscriptions) { + subscription.notifyUpdate(); + } + } + } + } + } + finally { + conn.close(); + } + } + catch (Exception e) { + // The getNotifications method does not throw a meaningful message on interruption. + // Therefore, we do not log an error, unless it occurred while active. + if (isActive()) { + LOGGER.error(e, "Failed to poll notifications from Postgres database"); + } + } + } + } + finally { + this.latch.countDown(); + } + + } + private boolean isActive() { if (Thread.interrupted()) { Thread.currentThread().interrupt(); diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java index b3b999880a1..6a9a0249e14 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv3ClientManager.java @@ -16,9 +16,6 @@ package org.springframework.integration.mqtt.core; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - import org.eclipse.paho.client.mqttv3.IMqttAsyncClient; import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttAsyncClient; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java index 6298a803b0b..a89b34aa609 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/core/Mqttv5ClientManager.java @@ -16,9 +16,6 @@ package org.springframework.integration.mqtt.core; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - import org.eclipse.paho.mqttv5.client.IMqttAsyncClient; import org.eclipse.paho.mqttv5.client.IMqttToken; import org.eclipse.paho.mqttv5.client.MqttAsyncClient; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java index ff030838583..eca80f508d2 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java @@ -17,8 +17,6 @@ package org.springframework.integration.mqtt.inbound; import java.util.Arrays; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; import org.eclipse.paho.client.mqttv3.IMqttAsyncClient; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java index 6744f6d9180..1055875e2aa 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java @@ -18,8 +18,6 @@ import java.util.Arrays; import java.util.Map; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.stream.IntStream; import org.eclipse.paho.mqttv5.client.IMqttAsyncClient; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java index 5b35fb7e36b..317a9b9604a 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/outbound/MqttPahoMessageHandler.java @@ -16,9 +16,6 @@ package org.springframework.integration.mqtt.outbound; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - import org.eclipse.paho.client.mqttv3.IMqttAsyncClient; import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java index ced6fe3522e..b89db27cc31 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ClientWebSocketContainer.java @@ -21,8 +21,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.Lifecycle; import org.springframework.context.SmartLifecycle; diff --git a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java index 8f862d9374e..fb103b8aa36 100644 --- a/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java +++ b/spring-integration-websocket/src/main/java/org/springframework/integration/websocket/ServerWebSocketContainer.java @@ -17,8 +17,6 @@ package org.springframework.integration.websocket; import java.util.Arrays; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.springframework.context.Lifecycle; import org.springframework.context.SmartLifecycle; diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java index 712c6e68eb8..c78efb7c3a2 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/config/LeaderInitiatorFactoryBean.java @@ -17,8 +17,6 @@ package org.springframework.integration.zookeeper.config; import java.util.UUID; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.apache.curator.framework.CuratorFramework; diff --git a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java index 574cf97080e..fece7d598f0 100644 --- a/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java +++ b/spring-integration-zookeeper/src/main/java/org/springframework/integration/zookeeper/lock/ZookeeperLockRegistry.java @@ -152,7 +152,8 @@ public Lock obtain(Object lockKey) { this.locksLock.lock(); try { lock = this.locks.computeIfAbsent(path, p -> new ZkLock(this.client, this.mutexTaskExecutor, p)); - } finally { + } + finally { this.locksLock.unlock(); } if (this.trackingTime) { From 8b04ec21f450eacef7997965e2124b807182eccc Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Sat, 17 Jun 2023 09:34:00 +0200 Subject: [PATCH 08/10] fix the mqtt parent/child lock monitor sharing --- ...stractMqttMessageDrivenChannelAdapter.java | 26 +++++++++---------- .../MqttPahoMessageDrivenChannelAdapter.java | 18 ++++++++----- ...Mqttv5PahoMessageDrivenChannelAdapter.java | 23 +++++++++------- 3 files changed, 37 insertions(+), 30 deletions(-) diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java index 243f1e2449b..6712b00687d 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/AbstractMqttMessageDrivenChannelAdapter.java @@ -58,7 +58,7 @@ public abstract class AbstractMqttMessageDrivenChannelAdapter extends MessageProducerSupport implements ApplicationEventPublisherAware, ClientManager.ConnectCallback { - protected final Lock lock = new ReentrantLock(); // NOSONAR + protected final Lock topicLock = new ReentrantLock(); // NOSONAR private final String url; @@ -137,7 +137,7 @@ public void setQos(int... qos) { @ManagedAttribute public int[] getQos() { - this.lock.lock(); + this.topicLock.lock(); try { int[] topicQos = new int[this.topics.size()]; int n = 0; @@ -147,7 +147,7 @@ public int[] getQos() { return topicQos; } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @@ -167,12 +167,12 @@ protected MqttMessageConverter getConverter() { @ManagedAttribute public String[] getTopic() { - this.lock.lock(); + this.topicLock.lock(); try { return this.topics.keySet().toArray(new String[0]); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @@ -256,7 +256,7 @@ protected long getCompletionTimeout() { */ @ManagedOperation public void addTopic(String topic, int qos) { - this.lock.lock(); + this.topicLock.lock(); try { if (this.topics.containsKey(topic)) { throw new MessagingException("Topic '" + topic + "' is already subscribed."); @@ -265,7 +265,7 @@ public void addTopic(String topic, int qos) { logger.debug(LogMessage.format("Added '%s' to subscriptions.", topic)); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @@ -278,14 +278,14 @@ public void addTopic(String topic, int qos) { @ManagedOperation public void addTopic(String... topic) { Assert.notNull(topic, "'topic' cannot be null"); - this.lock.lock(); + this.topicLock.lock(); try { for (String t : topic) { addTopic(t, 1); } } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @@ -301,7 +301,7 @@ public void addTopics(String[] topic, int[] qos) { Assert.notNull(topic, "'topic' cannot be null."); Assert.noNullElements(topic, "'topic' cannot contain any null elements."); Assert.isTrue(topic.length == qos.length, "topic and qos arrays must the be the same length."); - this.lock.lock(); + this.topicLock.lock(); try { for (String newTopic : topic) { if (this.topics.containsKey(newTopic)) { @@ -313,7 +313,7 @@ public void addTopics(String[] topic, int[] qos) { } } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @@ -325,7 +325,7 @@ public void addTopics(String[] topic, int[] qos) { */ @ManagedOperation public void removeTopic(String... topic) { - this.lock.lock(); + this.topicLock.lock(); try { for (String name : topic) { if (this.topics.remove(name) != null) { @@ -334,7 +334,7 @@ public void removeTopic(String... topic) { } } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java index eca80f508d2..d7fd809e4ed 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java @@ -17,6 +17,8 @@ package org.springframework.integration.mqtt.inbound; import java.util.Arrays; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Stream; import org.eclipse.paho.client.mqttv3.IMqttAsyncClient; @@ -56,7 +58,6 @@ * @author Gary Russell * @author Artem Bilan * @author Artem Vozhdayenko - * @author Christian Tzolov * * @since 4.0 * @@ -65,6 +66,9 @@ public class MqttPahoMessageDrivenChannelAdapter extends AbstractMqttMessageDrivenChannelAdapter implements MqttCallbackExtended, MqttPahoComponent { + + private final Lock lock = new ReentrantLock(); + private final MqttPahoClientFactory clientFactory; private volatile IMqttAsyncClient client; @@ -250,7 +254,7 @@ public void destroy() { @Override public void addTopic(String topic, int qos) { - this.lock.lock(); + this.topicLock.lock(); try { super.addTopic(topic, qos); if (this.client != null && this.client.isConnected()) { @@ -263,13 +267,13 @@ public void addTopic(String topic, int qos) { throw new MessagingException("Failed to subscribe to topic " + topic, e); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @Override public void removeTopic(String... topic) { - this.lock.lock(); + this.topicLock.lock(); try { if (this.client != null && this.client.isConnected()) { this.client.unsubscribe(topic).waitForCompletion(getCompletionTimeout()); @@ -280,12 +284,12 @@ public void removeTopic(String... topic) { throw new MessagingException("Failed to unsubscribe from topic(s) " + Arrays.toString(topic), e); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } private void subscribe() { - this.lock.lock(); + this.topicLock.lock(); String[] topics = getTopic(); ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); try { @@ -312,7 +316,7 @@ private void subscribe() { logger.error(ex, () -> "Error subscribing to " + Arrays.toString(topics)); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } if (this.client.isConnected()) { String message = "Connected and subscribed to " + Arrays.toString(topics); diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java index 1055875e2aa..89b34ddee10 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java @@ -18,6 +18,8 @@ import java.util.Arrays; import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.IntStream; import org.eclipse.paho.mqttv5.client.IMqttAsyncClient; @@ -73,7 +75,6 @@ * @author Lucas Bowler * @author Artem Vozhdayenko * @author Matthias Thoma - * @author Christian Tzolov * * @since 5.5.5 * @@ -82,6 +83,8 @@ public class Mqttv5PahoMessageDrivenChannelAdapter extends AbstractMqttMessageDrivenChannelAdapter implements MqttCallback, MqttComponent { + private final Lock lock = new ReentrantLock(); + private final MqttConnectionOptions connectionOptions; private IMqttAsyncClient mqttClient; @@ -213,7 +216,7 @@ protected void doStart() { } private void connect() throws MqttException { - this.lock.lock(); + this.lock.unlock(); try { var clientManager = getClientManager(); if (clientManager == null) { @@ -230,7 +233,7 @@ private void connect() throws MqttException { @Override protected void doStop() { - this.lock.lock(); + this.topicLock.lock(); this.readyToSubscribeOnStart = false; String[] topics = getTopic(); try { @@ -250,7 +253,7 @@ protected void doStop() { logger.error(ex, () -> "Error unsubscribing from " + Arrays.toString(topics)); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @@ -269,7 +272,7 @@ public void destroy() { @Override public void addTopic(String topic, int qos) { - this.lock.lock(); + this.topicLock.lock(); try { super.addTopic(topic, qos); if (this.mqttClient != null && this.mqttClient.isConnected()) { @@ -281,13 +284,13 @@ public void addTopic(String topic, int qos) { throw new MessagingException("Failed to subscribe to topic " + topic, ex); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @Override public void removeTopic(String... topic) { - this.lock.lock(); + this.topicLock.lock(); try { if (this.mqttClient != null && this.mqttClient.isConnected()) { this.mqttClient.unsubscribe(topic).waitForCompletion(getCompletionTimeout()); @@ -298,7 +301,7 @@ public void removeTopic(String... topic) { throw new MessagingException("Failed to unsubscribe from topic(s) " + Arrays.toString(topic), ex); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } @@ -391,7 +394,7 @@ private void subscribe() { String[] topics = getTopic(); ApplicationEventPublisher applicationEventPublisher = getApplicationEventPublisher(); - this.lock.lock(); + this.topicLock.lock(); try { if (topics.length == 0) { return; @@ -420,7 +423,7 @@ private void subscribe() { logger.error(ex, () -> "Error subscribing to " + Arrays.toString(topics)); } finally { - this.lock.unlock(); + this.topicLock.unlock(); } } From 708a141f0b413154c3b963242b1dbee5605eea7e Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Sat, 17 Jun 2023 09:48:55 +0200 Subject: [PATCH 09/10] fix the mqtt parent/child lock monitor sharing, v2 --- .../mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java | 1 - .../mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java index d7fd809e4ed..f28df29eb99 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/MqttPahoMessageDrivenChannelAdapter.java @@ -66,7 +66,6 @@ public class MqttPahoMessageDrivenChannelAdapter extends AbstractMqttMessageDrivenChannelAdapter implements MqttCallbackExtended, MqttPahoComponent { - private final Lock lock = new ReentrantLock(); private final MqttPahoClientFactory clientFactory; diff --git a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java index 89b34ddee10..b589da35fc6 100644 --- a/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java +++ b/spring-integration-mqtt/src/main/java/org/springframework/integration/mqtt/inbound/Mqttv5PahoMessageDrivenChannelAdapter.java @@ -216,7 +216,7 @@ protected void doStart() { } private void connect() throws MqttException { - this.lock.unlock(); + this.lock.lock(); try { var clientManager = getClientManager(); if (clientManager == null) { From 17b3894ebd94574e32b9ac73acbd130d10f8ee04 Mon Sep 17 00:00:00 2001 From: Christian Tzolov Date: Mon, 19 Jun 2023 15:39:37 +0200 Subject: [PATCH 10/10] patch the stomp test --- .../integration/stomp/AbstractStompSessionManager.java | 6 +++--- ...StompInboundChannelAdapterWebSocketIntegrationTests.java | 6 ++++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java index fa7e9c8e3c6..13ae624efcd 100644 --- a/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java +++ b/spring-integration-stomp/src/main/java/org/springframework/integration/stomp/AbstractStompSessionManager.java @@ -81,12 +81,12 @@ public abstract class AbstractStompSessionManager implements StompSessionManager protected final StompClientSupport stompClient; // NOSONAR final - private final Lock lock = new ReentrantLock(); - private final CompositeStompSessionHandler compositeStompSessionHandler = new CompositeStompSessionHandler(); private final Lock lifecycleMonitor = new ReentrantLock(); + private final Lock lock = new ReentrantLock(); + private final AtomicInteger epoch = new AtomicInteger(); private boolean autoStartup = false; @@ -390,7 +390,7 @@ void addHandler(StompSessionHandler delegate) { this.delegates.add(delegate); } finally { - this.delegatesMonitor.lock(); + this.delegatesMonitor.unlock(); } } diff --git a/spring-integration-stomp/src/test/java/org/springframework/integration/stomp/inbound/StompInboundChannelAdapterWebSocketIntegrationTests.java b/spring-integration-stomp/src/test/java/org/springframework/integration/stomp/inbound/StompInboundChannelAdapterWebSocketIntegrationTests.java index d10ca04b148..8d11caa1006 100644 --- a/spring-integration-stomp/src/test/java/org/springframework/integration/stomp/inbound/StompInboundChannelAdapterWebSocketIntegrationTests.java +++ b/spring-integration-stomp/src/test/java/org/springframework/integration/stomp/inbound/StompInboundChannelAdapterWebSocketIntegrationTests.java @@ -1,5 +1,5 @@ /* - * Copyright 2015-2022 the original author or authors. + * Copyright 2015-2023 the original author or authors. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -247,7 +247,8 @@ public WebSocketClient webSocketClient() { } @Bean - public WebSocketStompClient stompClient(TaskScheduler taskScheduler) { + public WebSocketStompClient stompClient( + @Qualifier("taskScheduler") TaskScheduler taskScheduler) { WebSocketStompClient webSocketStompClient = new WebSocketStompClient(webSocketClient()); webSocketStompClient.setMessageConverter(new MappingJackson2MessageConverter()); webSocketStompClient.setTaskScheduler(taskScheduler); @@ -347,6 +348,7 @@ public void configureMessageBroker(MessageBrokerRegistry configurer) { //SimpleBrokerMessageHandler doesn't support RECEIPT frame, hence we emulate it this way @Bean public ApplicationListener webSocketEventListener( + @Qualifier("clientOutboundChannel") final AbstractSubscribableChannel clientOutboundChannel) { return event -> { Message message = event.getMessage();