mirror of https://github.com/apache/nifi.git
NIFI-4834: Updated AbstractJMSProcessor to use a separate SingleConnectionFactory per concurrent task instead of sharing one across the entire processor.
This closes #2445. Signed-off-by: Andy LoPresto <alopresto@apache.org>
This commit is contained in:
parent
b5938062a8
commit
3ca7c3e7a1
|
@ -18,9 +18,13 @@ package org.apache.nifi.jms.processors;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import javax.jms.ConnectionFactory;
|
import javax.jms.ConnectionFactory;
|
||||||
|
|
||||||
|
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||||
import org.apache.nifi.components.PropertyDescriptor;
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
|
import org.apache.nifi.jms.cf.JMSConnectionFactoryProvider;
|
||||||
|
@ -38,9 +42,7 @@ import org.springframework.jms.core.JmsTemplate;
|
||||||
/**
|
/**
|
||||||
* Base JMS processor to support implementation of JMS producers and consumers.
|
* Base JMS processor to support implementation of JMS producers and consumers.
|
||||||
*
|
*
|
||||||
* @param <T>
|
* @param <T> the type of {@link JMSWorker} which could be {@link JMSPublisher} or {@link JMSConsumer}
|
||||||
* the type of {@link JMSWorker} which could be {@link JMSPublisher}
|
|
||||||
* or {@link JMSConsumer}
|
|
||||||
* @see PublishJMS
|
* @see PublishJMS
|
||||||
* @see ConsumeJMS
|
* @see ConsumeJMS
|
||||||
* @see JMSConnectionFactoryProviderDefinition
|
* @see JMSConnectionFactoryProviderDefinition
|
||||||
|
@ -48,7 +50,6 @@ import org.springframework.jms.core.JmsTemplate;
|
||||||
abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcessor {
|
abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcessor {
|
||||||
|
|
||||||
static final String QUEUE = "QUEUE";
|
static final String QUEUE = "QUEUE";
|
||||||
|
|
||||||
static final String TOPIC = "TOPIC";
|
static final String TOPIC = "TOPIC";
|
||||||
|
|
||||||
static final PropertyDescriptor USER = new PropertyDescriptor.Builder()
|
static final PropertyDescriptor USER = new PropertyDescriptor.Builder()
|
||||||
|
@ -90,14 +91,13 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
|
||||||
.build();
|
.build();
|
||||||
static final PropertyDescriptor SESSION_CACHE_SIZE = new PropertyDescriptor.Builder()
|
static final PropertyDescriptor SESSION_CACHE_SIZE = new PropertyDescriptor.Builder()
|
||||||
.name("Session Cache size")
|
.name("Session Cache size")
|
||||||
.description("The maximum limit for the number of cached Sessions.")
|
.description("This property is deprecated and no longer has any effect on the Processor. It will be removed in a later version.")
|
||||||
.required(true)
|
.required(false)
|
||||||
.defaultValue("1")
|
.defaultValue("1")
|
||||||
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
|
||||||
// ConnectionFactoryProvider ControllerService
|
|
||||||
static final PropertyDescriptor CF_SERVICE = new PropertyDescriptor.Builder()
|
static final PropertyDescriptor CF_SERVICE = new PropertyDescriptor.Builder()
|
||||||
.name("Connection Factory Service")
|
.name("Connection Factory Service")
|
||||||
.description("The Controller Service that is used to obtain ConnectionFactory")
|
.description("The Controller Service that is used to obtain ConnectionFactory")
|
||||||
|
@ -106,11 +106,9 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
static final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>();
|
static final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>();
|
||||||
|
private volatile BlockingQueue<T> workerPool;
|
||||||
|
private final AtomicInteger clientIdCounter = new AtomicInteger(1);
|
||||||
|
|
||||||
/*
|
|
||||||
* Will ensure that list of PropertyDescriptors is build only once, since
|
|
||||||
* all other lifecycle methods are invoked multiple times.
|
|
||||||
*/
|
|
||||||
static {
|
static {
|
||||||
propertyDescriptors.add(CF_SERVICE);
|
propertyDescriptors.add(CF_SERVICE);
|
||||||
propertyDescriptors.add(DESTINATION);
|
propertyDescriptors.add(DESTINATION);
|
||||||
|
@ -121,47 +119,35 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
|
||||||
propertyDescriptors.add(SESSION_CACHE_SIZE);
|
propertyDescriptors.add(SESSION_CACHE_SIZE);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected volatile T targetResource;
|
|
||||||
|
|
||||||
private volatile CachingConnectionFactory cachingConnectionFactory;
|
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
@Override
|
@Override
|
||||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||||
return propertyDescriptors;
|
return propertyDescriptors;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Builds target resource ({@link JMSPublisher} or {@link JMSConsumer}) upon
|
|
||||||
* first invocation while delegating to the sub-classes ( {@link PublishJMS}
|
|
||||||
* or {@link ConsumeJMS}) via
|
|
||||||
* {@link #rendezvousWithJms(ProcessContext, ProcessSession)} method.
|
|
||||||
*/
|
|
||||||
@Override
|
@Override
|
||||||
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
|
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||||
synchronized (this) {
|
T worker = workerPool.poll();
|
||||||
this.buildTargetResource(context);
|
if (worker == null) {
|
||||||
|
worker = buildTargetResource(context);
|
||||||
}
|
}
|
||||||
this.rendezvousWithJms(context, session);
|
|
||||||
|
rendezvousWithJms(context, session, worker);
|
||||||
|
workerPool.offer(worker);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@OnScheduled
|
||||||
* Will destroy the instance of {@link CachingConnectionFactory} and sets
|
public void setupWorkerPool(final ProcessContext context) {
|
||||||
* 'targetResource' to null;
|
workerPool = new LinkedBlockingQueue<>(context.getMaxConcurrentTasks());
|
||||||
*/
|
}
|
||||||
|
|
||||||
|
|
||||||
@OnStopped
|
@OnStopped
|
||||||
public void close() {
|
public void close() {
|
||||||
if (this.cachingConnectionFactory != null) {
|
T worker;
|
||||||
this.cachingConnectionFactory.destroy();
|
while ((worker = workerPool.poll()) != null) {
|
||||||
|
worker.shutdown();
|
||||||
}
|
}
|
||||||
this.targetResource = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return this.getClass().getSimpleName() + " - " + this.targetResource;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -169,23 +155,16 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
|
||||||
* {@link #onTrigger(ProcessContext, ProcessSession)} operation. It is
|
* {@link #onTrigger(ProcessContext, ProcessSession)} operation. It is
|
||||||
* implemented by sub-classes to perform {@link Processor} specific
|
* implemented by sub-classes to perform {@link Processor} specific
|
||||||
* functionality.
|
* functionality.
|
||||||
*
|
|
||||||
* @param context
|
|
||||||
* instance of {@link ProcessContext}
|
|
||||||
* @param session
|
|
||||||
* instance of {@link ProcessSession}
|
|
||||||
*/
|
*/
|
||||||
protected abstract void rendezvousWithJms(ProcessContext context, ProcessSession session) throws ProcessException;
|
protected abstract void rendezvousWithJms(ProcessContext context, ProcessSession session, T jmsWorker) throws ProcessException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Finishes building one of the {@link JMSWorker} subclasses T.
|
* Finishes building one of the {@link JMSWorker} subclasses T.
|
||||||
*
|
*
|
||||||
* @param jmsTemplate instance of {@link JmsTemplate}
|
|
||||||
*
|
|
||||||
* @see JMSPublisher
|
* @see JMSPublisher
|
||||||
* @see JMSConsumer
|
* @see JMSConsumer
|
||||||
*/
|
*/
|
||||||
protected abstract T finishBuildingTargetResource(JmsTemplate jmsTemplate, ProcessContext processContext);
|
protected abstract T finishBuildingJmsWorker(CachingConnectionFactory connectionFactory, JmsTemplate jmsTemplate, ProcessContext processContext);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method essentially performs initialization of this Processor by
|
* This method essentially performs initialization of this Processor by
|
||||||
|
@ -195,30 +174,30 @@ abstract class AbstractJMSProcessor<T extends JMSWorker> extends AbstractProcess
|
||||||
* in an instance of the {@link CachingConnectionFactory} used to construct
|
* in an instance of the {@link CachingConnectionFactory} used to construct
|
||||||
* {@link JmsTemplate} used by this Processor.
|
* {@link JmsTemplate} used by this Processor.
|
||||||
*/
|
*/
|
||||||
private void buildTargetResource(ProcessContext context) {
|
private T buildTargetResource(ProcessContext context) {
|
||||||
if (this.targetResource == null) {
|
final JMSConnectionFactoryProviderDefinition cfProvider = context.getProperty(CF_SERVICE).asControllerService(JMSConnectionFactoryProviderDefinition.class);
|
||||||
JMSConnectionFactoryProviderDefinition cfProvider = context.getProperty(CF_SERVICE).asControllerService(JMSConnectionFactoryProviderDefinition.class);
|
final ConnectionFactory connectionFactory = cfProvider.getConnectionFactory();
|
||||||
ConnectionFactory connectionFactory = cfProvider.getConnectionFactory();
|
|
||||||
|
|
||||||
UserCredentialsConnectionFactoryAdapter cfCredentialsAdapter = new UserCredentialsConnectionFactoryAdapter();
|
final UserCredentialsConnectionFactoryAdapter cfCredentialsAdapter = new UserCredentialsConnectionFactoryAdapter();
|
||||||
cfCredentialsAdapter.setTargetConnectionFactory(connectionFactory);
|
cfCredentialsAdapter.setTargetConnectionFactory(connectionFactory);
|
||||||
cfCredentialsAdapter.setUsername(context.getProperty(USER).getValue());
|
cfCredentialsAdapter.setUsername(context.getProperty(USER).getValue());
|
||||||
cfCredentialsAdapter.setPassword(context.getProperty(PASSWORD).getValue());
|
cfCredentialsAdapter.setPassword(context.getProperty(PASSWORD).getValue());
|
||||||
|
|
||||||
this.cachingConnectionFactory = new CachingConnectionFactory(cfCredentialsAdapter);
|
final CachingConnectionFactory cachingFactory = new CachingConnectionFactory(cfCredentialsAdapter);
|
||||||
this.cachingConnectionFactory.setSessionCacheSize(Integer.parseInt(context.getProperty(SESSION_CACHE_SIZE).getValue()));
|
|
||||||
String clientId = context.getProperty(CLIENT_ID).evaluateAttributeExpressions().getValue();
|
|
||||||
if (clientId != null) {
|
|
||||||
this.cachingConnectionFactory.setClientId(clientId);
|
|
||||||
}
|
|
||||||
JmsTemplate jmsTemplate = new JmsTemplate();
|
|
||||||
jmsTemplate.setConnectionFactory(this.cachingConnectionFactory);
|
|
||||||
jmsTemplate.setPubSubDomain(TOPIC.equals(context.getProperty(DESTINATION_TYPE).getValue()));
|
|
||||||
|
|
||||||
// set of properties that may be good candidates for exposure via configuration
|
String clientId = context.getProperty(CLIENT_ID).evaluateAttributeExpressions().getValue();
|
||||||
jmsTemplate.setReceiveTimeout(1000);
|
if (clientId != null) {
|
||||||
|
clientId = clientId + "-" + clientIdCounter.getAndIncrement();
|
||||||
this.targetResource = this.finishBuildingTargetResource(jmsTemplate, context);
|
cachingFactory.setClientId(clientId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
JmsTemplate jmsTemplate = new JmsTemplate();
|
||||||
|
jmsTemplate.setConnectionFactory(cachingFactory);
|
||||||
|
jmsTemplate.setPubSubDomain(TOPIC.equals(context.getProperty(DESTINATION_TYPE).getValue()));
|
||||||
|
|
||||||
|
// set of properties that may be good candidates for exposure via configuration
|
||||||
|
jmsTemplate.setReceiveTimeout(1000);
|
||||||
|
|
||||||
|
return finishBuildingJmsWorker(cachingFactory, jmsTemplate, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,8 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.jms.processors;
|
package org.apache.nifi.jms.processors;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.OutputStream;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -44,8 +42,8 @@ import org.apache.nifi.processor.ProcessContext;
|
||||||
import org.apache.nifi.processor.ProcessSession;
|
import org.apache.nifi.processor.ProcessSession;
|
||||||
import org.apache.nifi.processor.Relationship;
|
import org.apache.nifi.processor.Relationship;
|
||||||
import org.apache.nifi.processor.exception.ProcessException;
|
import org.apache.nifi.processor.exception.ProcessException;
|
||||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
|
||||||
import org.apache.nifi.processor.util.StandardValidators;
|
import org.apache.nifi.processor.util.StandardValidators;
|
||||||
|
import org.springframework.jms.connection.CachingConnectionFactory;
|
||||||
import org.springframework.jms.core.JmsTemplate;
|
import org.springframework.jms.core.JmsTemplate;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -146,35 +144,34 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
|
||||||
* 'success' {@link Relationship}.
|
* 'success' {@link Relationship}.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected void rendezvousWithJms(final ProcessContext context, final ProcessSession processSession) throws ProcessException {
|
protected void rendezvousWithJms(final ProcessContext context, final ProcessSession processSession, final JMSConsumer consumer) throws ProcessException {
|
||||||
final String destinationName = context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
|
final String destinationName = context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue();
|
||||||
final Boolean durableBoolean = context.getProperty(DURABLE_SUBSCRIBER).evaluateAttributeExpressions().asBoolean();
|
final Boolean durableBoolean = context.getProperty(DURABLE_SUBSCRIBER).evaluateAttributeExpressions().asBoolean();
|
||||||
final boolean durable = durableBoolean == null ? false : durableBoolean;
|
final boolean durable = durableBoolean == null ? false : durableBoolean;
|
||||||
final Boolean sharedBoolean = context.getProperty(SHARED_SUBSCRIBER).evaluateAttributeExpressions().asBoolean();
|
final Boolean sharedBoolean = context.getProperty(SHARED_SUBSCRIBER).evaluateAttributeExpressions().asBoolean();
|
||||||
final boolean shared = sharedBoolean == null ? false : sharedBoolean;
|
final boolean shared = sharedBoolean == null ? false : sharedBoolean;
|
||||||
final String subscriptionName = context.getProperty(SUBSCRIPTION_NAME).evaluateAttributeExpressions().getValue();
|
final String subscriptionName = context.getProperty(SUBSCRIPTION_NAME).evaluateAttributeExpressions().getValue();
|
||||||
this.targetResource.consume(destinationName, durable, shared, subscriptionName, new ConsumerCallback(){
|
|
||||||
|
consumer.consume(destinationName, durable, shared, subscriptionName, new ConsumerCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void accept(final JMSResponse response) {
|
public void accept(final JMSResponse response) {
|
||||||
if (response != null){
|
if (response == null) {
|
||||||
FlowFile flowFile = processSession.create();
|
return;
|
||||||
flowFile = processSession.write(flowFile, new OutputStreamCallback() {
|
|
||||||
@Override
|
|
||||||
public void process(final OutputStream out) throws IOException {
|
|
||||||
out.write(response.getMessageBody());
|
|
||||||
}
|
|
||||||
});
|
|
||||||
Map<String, Object> jmsHeaders = response.getMessageHeaders();
|
|
||||||
Map<String, Object> jmsProperties = Collections.<String, Object>unmodifiableMap(response.getMessageProperties());
|
|
||||||
flowFile = ConsumeJMS.this.updateFlowFileAttributesWithJMSAttributes(jmsHeaders, flowFile, processSession);
|
|
||||||
flowFile = ConsumeJMS.this.updateFlowFileAttributesWithJMSAttributes(jmsProperties, flowFile, processSession);
|
|
||||||
flowFile = processSession.putAttribute(flowFile, JMS_SOURCE_DESTINATION_NAME, destinationName);
|
|
||||||
processSession.getProvenanceReporter().receive(flowFile, destinationName);
|
|
||||||
processSession.transfer(flowFile, REL_SUCCESS);
|
|
||||||
processSession.commit();
|
|
||||||
} else {
|
|
||||||
context.yield();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
FlowFile flowFile = processSession.create();
|
||||||
|
flowFile = processSession.write(flowFile, out -> out.write(response.getMessageBody()));
|
||||||
|
|
||||||
|
final Map<String, String> jmsHeaders = response.getMessageHeaders();
|
||||||
|
final Map<String, String> jmsProperties = response.getMessageProperties();
|
||||||
|
|
||||||
|
flowFile = ConsumeJMS.this.updateFlowFileAttributesWithJMSAttributes(jmsHeaders, flowFile, processSession);
|
||||||
|
flowFile = ConsumeJMS.this.updateFlowFileAttributesWithJMSAttributes(jmsProperties, flowFile, processSession);
|
||||||
|
flowFile = processSession.putAttribute(flowFile, JMS_SOURCE_DESTINATION_NAME, destinationName);
|
||||||
|
|
||||||
|
processSession.getProvenanceReporter().receive(flowFile, destinationName);
|
||||||
|
processSession.transfer(flowFile, REL_SUCCESS);
|
||||||
|
processSession.commit();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -183,23 +180,17 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
|
||||||
* Will create an instance of {@link JMSConsumer}
|
* Will create an instance of {@link JMSConsumer}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected JMSConsumer finishBuildingTargetResource(JmsTemplate jmsTemplate, ProcessContext processContext) {
|
protected JMSConsumer finishBuildingJmsWorker(CachingConnectionFactory connectionFactory, JmsTemplate jmsTemplate, ProcessContext processContext) {
|
||||||
int ackMode = processContext.getProperty(ACKNOWLEDGEMENT_MODE).asInteger();
|
int ackMode = processContext.getProperty(ACKNOWLEDGEMENT_MODE).asInteger();
|
||||||
jmsTemplate.setSessionAcknowledgeMode(ackMode);
|
jmsTemplate.setSessionAcknowledgeMode(ackMode);
|
||||||
return new JMSConsumer(jmsTemplate, this.getLogger());
|
return new JMSConsumer(connectionFactory, jmsTemplate, this.getLogger());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
@Override
|
@Override
|
||||||
public Set<Relationship> getRelationships() {
|
public Set<Relationship> getRelationships() {
|
||||||
return relationships;
|
return relationships;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
@Override
|
@Override
|
||||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||||
return thisPropertyDescriptors;
|
return thisPropertyDescriptors;
|
||||||
|
@ -211,11 +202,12 @@ public class ConsumeJMS extends AbstractJMSProcessor<JMSConsumer> {
|
||||||
* copied values of JMS attributes will be "stringified" via
|
* copied values of JMS attributes will be "stringified" via
|
||||||
* String.valueOf(attribute).
|
* String.valueOf(attribute).
|
||||||
*/
|
*/
|
||||||
private FlowFile updateFlowFileAttributesWithJMSAttributes(Map<String, Object> jmsAttributes, FlowFile flowFile, ProcessSession processSession) {
|
private FlowFile updateFlowFileAttributesWithJMSAttributes(Map<String, String> jmsAttributes, FlowFile flowFile, ProcessSession processSession) {
|
||||||
Map<String, String> attributes = new HashMap<String, String>();
|
Map<String, String> attributes = new HashMap<>();
|
||||||
for (Entry<String, Object> entry : jmsAttributes.entrySet()) {
|
for (Entry<String, String> entry : jmsAttributes.entrySet()) {
|
||||||
attributes.put(entry.getKey(), String.valueOf(entry.getValue()));
|
attributes.put(entry.getKey(), entry.getValue());
|
||||||
}
|
}
|
||||||
|
|
||||||
flowFile = processSession.putAllAttributes(flowFile, attributes);
|
flowFile = processSession.putAllAttributes(flowFile, attributes);
|
||||||
return flowFile;
|
return flowFile;
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,7 @@ import javax.jms.Topic;
|
||||||
|
|
||||||
import org.apache.nifi.logging.ComponentLog;
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
import org.apache.nifi.processor.exception.ProcessException;
|
import org.apache.nifi.processor.exception.ProcessException;
|
||||||
|
import org.springframework.jms.connection.CachingConnectionFactory;
|
||||||
import org.springframework.jms.core.JmsTemplate;
|
import org.springframework.jms.core.JmsTemplate;
|
||||||
import org.springframework.jms.core.SessionCallback;
|
import org.springframework.jms.core.SessionCallback;
|
||||||
import org.springframework.jms.support.JmsHeaders;
|
import org.springframework.jms.support.JmsHeaders;
|
||||||
|
@ -43,63 +44,54 @@ import org.springframework.jms.support.JmsUtils;
|
||||||
*/
|
*/
|
||||||
final class JMSConsumer extends JMSWorker {
|
final class JMSConsumer extends JMSWorker {
|
||||||
|
|
||||||
/**
|
JMSConsumer(CachingConnectionFactory connectionFactory, JmsTemplate jmsTemplate, ComponentLog logger) {
|
||||||
* Creates an instance of this consumer
|
super(connectionFactory, jmsTemplate, logger);
|
||||||
*
|
logger.debug("Created Message Consumer for '{}'", new Object[] {jmsTemplate});
|
||||||
* @param jmsTemplate
|
}
|
||||||
* instance of {@link JmsTemplate}
|
|
||||||
* @param processLog
|
|
||||||
* instance of {@link ComponentLog}
|
private MessageConsumer createMessageConsumer(final Session session, final String destinationName, final boolean durable, final boolean shared, final String subscriberName) throws JMSException {
|
||||||
*/
|
final boolean isPubSub = JMSConsumer.this.jmsTemplate.isPubSubDomain();
|
||||||
JMSConsumer(JmsTemplate jmsTemplate, ComponentLog processLog) {
|
final Destination destination = JMSConsumer.this.jmsTemplate.getDestinationResolver().resolveDestinationName(session, destinationName, isPubSub);
|
||||||
super(jmsTemplate, processLog);
|
|
||||||
if (this.processLog.isInfoEnabled()) {
|
if (isPubSub) {
|
||||||
this.processLog.info("Created Message Consumer for '" + jmsTemplate.toString() + "'.");
|
if (shared) {
|
||||||
|
try {
|
||||||
|
if (durable) {
|
||||||
|
return session.createSharedDurableConsumer((Topic) destination, subscriberName);
|
||||||
|
} else {
|
||||||
|
return session.createSharedConsumer((Topic) destination, subscriberName);
|
||||||
|
}
|
||||||
|
} catch (AbstractMethodError e) {
|
||||||
|
throw new ProcessException("Failed to create a shared consumer. Make sure the target broker is JMS 2.0 compliant.", e);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (durable) {
|
||||||
|
return session.createDurableConsumer((Topic) destination, subscriberName, null, JMSConsumer.this.jmsTemplate.isPubSubDomain());
|
||||||
|
} else {
|
||||||
|
return session.createConsumer(destination, null, JMSConsumer.this.jmsTemplate.isPubSubDomain());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return session.createConsumer(destination, null, JMSConsumer.this.jmsTemplate.isPubSubDomain());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
public void consume(final String destinationName, final boolean durable, final boolean shared, final String subscriberName, final ConsumerCallback consumerCallback) {
|
public void consume(final String destinationName, final boolean durable, final boolean shared, final String subscriberName, final ConsumerCallback consumerCallback) {
|
||||||
this.jmsTemplate.execute(new SessionCallback<Void>() {
|
this.jmsTemplate.execute(new SessionCallback<Void>() {
|
||||||
@Override
|
@Override
|
||||||
public Void doInJms(Session session) throws JMSException {
|
public Void doInJms(final Session session) throws JMSException {
|
||||||
/*
|
// We need to call recover to ensure that in in the event of
|
||||||
* We need to call recover to ensure that in in the event of
|
// abrupt end or exception the current session will stop message
|
||||||
* abrupt end or exception the current session will stop message
|
// delivery and restarts with the oldest unacknowledged message
|
||||||
* delivery and restarts with the oldest unacknowledged message
|
|
||||||
*/
|
|
||||||
session.recover();
|
session.recover();
|
||||||
boolean isPubSub = JMSConsumer.this.jmsTemplate.isPubSubDomain();
|
|
||||||
Destination destination = JMSConsumer.this.jmsTemplate.getDestinationResolver().resolveDestinationName(
|
final MessageConsumer msgConsumer = createMessageConsumer(session, destinationName, durable, shared, subscriberName);
|
||||||
session, destinationName, isPubSub);
|
|
||||||
MessageConsumer msgConsumer;
|
|
||||||
if (isPubSub) {
|
|
||||||
if (shared) {
|
|
||||||
try {
|
|
||||||
if (durable) {
|
|
||||||
msgConsumer = session.createSharedDurableConsumer((Topic)destination, subscriberName);
|
|
||||||
} else {
|
|
||||||
msgConsumer = session.createSharedConsumer((Topic)destination, subscriberName);
|
|
||||||
}
|
|
||||||
} catch (AbstractMethodError e) {
|
|
||||||
throw new ProcessException("Failed to create a shared consumer. Make sure the target broker is JMS 2.0 compliant.", e);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
if (durable) {
|
|
||||||
msgConsumer = session.createDurableConsumer((Topic)destination, subscriberName, null, JMSConsumer.this.jmsTemplate.isPubSubDomain());
|
|
||||||
} else {
|
|
||||||
msgConsumer = session.createConsumer((Topic)destination, null, JMSConsumer.this.jmsTemplate.isPubSubDomain());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
msgConsumer = session.createConsumer(destination, null, JMSConsumer.this.jmsTemplate.isPubSubDomain());
|
|
||||||
}
|
|
||||||
Message message = msgConsumer.receive(JMSConsumer.this.jmsTemplate.getReceiveTimeout());
|
|
||||||
JMSResponse response = null;
|
|
||||||
try {
|
try {
|
||||||
|
final Message message = msgConsumer.receive(JMSConsumer.this.jmsTemplate.getReceiveTimeout());
|
||||||
|
JMSResponse response = null;
|
||||||
|
|
||||||
if (message != null) {
|
if (message != null) {
|
||||||
byte[] messageBody = null;
|
byte[] messageBody = null;
|
||||||
if (message instanceof TextMessage) {
|
if (message instanceof TextMessage) {
|
||||||
|
@ -108,12 +100,14 @@ final class JMSConsumer extends JMSWorker {
|
||||||
messageBody = MessageBodyToBytesConverter.toBytes((BytesMessage) message);
|
messageBody = MessageBodyToBytesConverter.toBytes((BytesMessage) message);
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalStateException("Message type other then TextMessage and BytesMessage are "
|
throw new IllegalStateException("Message type other then TextMessage and BytesMessage are "
|
||||||
+ "not supported at the moment");
|
+ "not supported at the moment");
|
||||||
}
|
}
|
||||||
Map<String, Object> messageHeaders = extractMessageHeaders(message);
|
|
||||||
Map<String, String> messageProperties = extractMessageProperties(message);
|
final Map<String, String> messageHeaders = extractMessageHeaders(message);
|
||||||
|
final Map<String, String> messageProperties = extractMessageProperties(message);
|
||||||
response = new JMSResponse(messageBody, messageHeaders, messageProperties);
|
response = new JMSResponse(messageBody, messageHeaders, messageProperties);
|
||||||
}
|
}
|
||||||
|
|
||||||
// invoke the processor callback (regardless if it's null,
|
// invoke the processor callback (regardless if it's null,
|
||||||
// so the processor can yield) as part of this inJMS call
|
// so the processor can yield) as part of this inJMS call
|
||||||
// and ACK message *only* after its successful invocation
|
// and ACK message *only* after its successful invocation
|
||||||
|
@ -125,19 +119,18 @@ final class JMSConsumer extends JMSWorker {
|
||||||
} finally {
|
} finally {
|
||||||
JmsUtils.closeMessageConsumer(msgConsumer);
|
JmsUtils.closeMessageConsumer(msgConsumer);
|
||||||
}
|
}
|
||||||
|
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}, true);
|
}, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private Map<String, String> extractMessageProperties(Message message) {
|
private Map<String, String> extractMessageProperties(final Message message) {
|
||||||
Map<String, String> properties = new HashMap<>();
|
final Map<String, String> properties = new HashMap<>();
|
||||||
try {
|
try {
|
||||||
Enumeration<String> propertyNames = message.getPropertyNames();
|
final Enumeration<String> propertyNames = message.getPropertyNames();
|
||||||
while (propertyNames.hasMoreElements()) {
|
while (propertyNames.hasMoreElements()) {
|
||||||
String propertyName = propertyNames.nextElement();
|
String propertyName = propertyNames.nextElement();
|
||||||
properties.put(propertyName, String.valueOf(message.getObjectProperty(propertyName)));
|
properties.put(propertyName, String.valueOf(message.getObjectProperty(propertyName)));
|
||||||
|
@ -148,41 +141,33 @@ final class JMSConsumer extends JMSWorker {
|
||||||
return properties;
|
return properties;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
private Map<String, Object> extractMessageHeaders(Message message) {
|
|
||||||
// even though all values are Strings in current impl, it may change in
|
|
||||||
// the future, so keeping it <String, Object>
|
|
||||||
Map<String, Object> messageHeaders = new HashMap<>();
|
|
||||||
try {
|
|
||||||
messageHeaders.put(JmsHeaders.DELIVERY_MODE, String.valueOf(message.getJMSDeliveryMode()));
|
|
||||||
messageHeaders.put(JmsHeaders.EXPIRATION, String.valueOf(message.getJMSExpiration()));
|
|
||||||
messageHeaders.put(JmsHeaders.PRIORITY, String.valueOf(message.getJMSPriority()));
|
|
||||||
messageHeaders.put(JmsHeaders.REDELIVERED, String.valueOf(message.getJMSRedelivered()));
|
|
||||||
messageHeaders.put(JmsHeaders.TIMESTAMP, String.valueOf(message.getJMSTimestamp()));
|
|
||||||
messageHeaders.put(JmsHeaders.CORRELATION_ID, message.getJMSCorrelationID());
|
|
||||||
messageHeaders.put(JmsHeaders.MESSAGE_ID, message.getJMSMessageID());
|
|
||||||
messageHeaders.put(JmsHeaders.TYPE, message.getJMSType());
|
|
||||||
|
|
||||||
String replyToDestinationName = this.retrieveDestinationName(message.getJMSReplyTo(), JmsHeaders.REPLY_TO);
|
private Map<String, String> extractMessageHeaders(final Message message) throws JMSException {
|
||||||
if (replyToDestinationName != null) {
|
final Map<String, String> messageHeaders = new HashMap<>();
|
||||||
messageHeaders.put(JmsHeaders.REPLY_TO, replyToDestinationName);
|
|
||||||
}
|
messageHeaders.put(JmsHeaders.DELIVERY_MODE, String.valueOf(message.getJMSDeliveryMode()));
|
||||||
String destinationName = this.retrieveDestinationName(message.getJMSDestination(), JmsHeaders.DESTINATION);
|
messageHeaders.put(JmsHeaders.EXPIRATION, String.valueOf(message.getJMSExpiration()));
|
||||||
if (destinationName != null) {
|
messageHeaders.put(JmsHeaders.PRIORITY, String.valueOf(message.getJMSPriority()));
|
||||||
messageHeaders.put(JmsHeaders.DESTINATION, destinationName);
|
messageHeaders.put(JmsHeaders.REDELIVERED, String.valueOf(message.getJMSRedelivered()));
|
||||||
}
|
messageHeaders.put(JmsHeaders.TIMESTAMP, String.valueOf(message.getJMSTimestamp()));
|
||||||
} catch (Exception e) {
|
messageHeaders.put(JmsHeaders.CORRELATION_ID, message.getJMSCorrelationID());
|
||||||
throw new IllegalStateException("Failed to extract JMS Headers", e);
|
messageHeaders.put(JmsHeaders.MESSAGE_ID, message.getJMSMessageID());
|
||||||
|
messageHeaders.put(JmsHeaders.TYPE, message.getJMSType());
|
||||||
|
|
||||||
|
String replyToDestinationName = this.retrieveDestinationName(message.getJMSReplyTo(), JmsHeaders.REPLY_TO);
|
||||||
|
if (replyToDestinationName != null) {
|
||||||
|
messageHeaders.put(JmsHeaders.REPLY_TO, replyToDestinationName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
String destinationName = this.retrieveDestinationName(message.getJMSDestination(), JmsHeaders.DESTINATION);
|
||||||
|
if (destinationName != null) {
|
||||||
|
messageHeaders.put(JmsHeaders.DESTINATION, destinationName);
|
||||||
|
}
|
||||||
|
|
||||||
return messageHeaders;
|
return messageHeaders;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
private String retrieveDestinationName(Destination destination, String headerName) {
|
private String retrieveDestinationName(Destination destination, String headerName) {
|
||||||
String destinationName = null;
|
String destinationName = null;
|
||||||
if (destination != null) {
|
if (destination != null) {
|
||||||
|
@ -196,17 +181,14 @@ final class JMSConsumer extends JMSWorker {
|
||||||
return destinationName;
|
return destinationName;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
static class JMSResponse {
|
static class JMSResponse {
|
||||||
private final byte[] messageBody;
|
private final byte[] messageBody;
|
||||||
|
|
||||||
private final Map<String, Object> messageHeaders;
|
private final Map<String, String> messageHeaders;
|
||||||
|
|
||||||
private final Map<String, String> messageProperties;
|
private final Map<String, String> messageProperties;
|
||||||
|
|
||||||
JMSResponse(byte[] messageBody, Map<String, Object> messageHeaders, Map<String, String> messageProperties) {
|
JMSResponse(byte[] messageBody, Map<String, String> messageHeaders, Map<String, String> messageProperties) {
|
||||||
this.messageBody = messageBody;
|
this.messageBody = messageBody;
|
||||||
this.messageHeaders = Collections.unmodifiableMap(messageHeaders);
|
this.messageHeaders = Collections.unmodifiableMap(messageHeaders);
|
||||||
this.messageProperties = Collections.unmodifiableMap(messageProperties);
|
this.messageProperties = Collections.unmodifiableMap(messageProperties);
|
||||||
|
@ -216,7 +198,7 @@ final class JMSConsumer extends JMSWorker {
|
||||||
return this.messageBody;
|
return this.messageBody;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<String, Object> getMessageHeaders() {
|
public Map<String, String> getMessageHeaders() {
|
||||||
return this.messageHeaders;
|
return this.messageHeaders;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -27,10 +27,8 @@ import javax.jms.Queue;
|
||||||
import javax.jms.Session;
|
import javax.jms.Session;
|
||||||
import javax.jms.Topic;
|
import javax.jms.Topic;
|
||||||
|
|
||||||
import org.apache.nifi.flowfile.FlowFile;
|
|
||||||
import org.apache.nifi.logging.ComponentLog;
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
import org.slf4j.Logger;
|
import org.springframework.jms.connection.CachingConnectionFactory;
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
import org.springframework.jms.core.JmsTemplate;
|
import org.springframework.jms.core.JmsTemplate;
|
||||||
import org.springframework.jms.core.MessageCreator;
|
import org.springframework.jms.core.MessageCreator;
|
||||||
import org.springframework.jms.core.SessionCallback;
|
import org.springframework.jms.core.SessionCallback;
|
||||||
|
@ -41,45 +39,22 @@ import org.springframework.jms.support.JmsHeaders;
|
||||||
*/
|
*/
|
||||||
final class JMSPublisher extends JMSWorker {
|
final class JMSPublisher extends JMSWorker {
|
||||||
|
|
||||||
private final static Logger logger = LoggerFactory.getLogger(JMSPublisher.class);
|
JMSPublisher(CachingConnectionFactory connectionFactory, JmsTemplate jmsTemplate, ComponentLog processLog) {
|
||||||
|
super(connectionFactory, jmsTemplate, processLog);
|
||||||
|
processLog.debug("Created Message Publisher for {}", new Object[] {jmsTemplate});
|
||||||
/**
|
|
||||||
* Creates an instance of this publisher
|
|
||||||
*
|
|
||||||
* @param jmsTemplate
|
|
||||||
* instance of {@link JmsTemplate}
|
|
||||||
* @param processLog
|
|
||||||
* instance of {@link ComponentLog}
|
|
||||||
*/
|
|
||||||
JMSPublisher(JmsTemplate jmsTemplate, ComponentLog processLog) {
|
|
||||||
super(jmsTemplate, processLog);
|
|
||||||
if (logger.isInfoEnabled()) {
|
|
||||||
logger.info("Created Message Publisher for '" + jmsTemplate.toString() + "'.");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* @param messageBytes byte array representing contents of the message
|
|
||||||
*/
|
|
||||||
void publish(String destinationName, byte[] messageBytes) {
|
void publish(String destinationName, byte[] messageBytes) {
|
||||||
this.publish(destinationName, messageBytes, null);
|
this.publish(destinationName, messageBytes, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* @param messageBytes
|
|
||||||
* byte array representing contents of the message
|
|
||||||
* @param flowFileAttributes
|
|
||||||
* Map representing {@link FlowFile} attributes.
|
|
||||||
*/
|
|
||||||
void publish(final String destinationName, final byte[] messageBytes, final Map<String, String> flowFileAttributes) {
|
void publish(final String destinationName, final byte[] messageBytes, final Map<String, String> flowFileAttributes) {
|
||||||
this.jmsTemplate.send(destinationName, new MessageCreator() {
|
this.jmsTemplate.send(destinationName, new MessageCreator() {
|
||||||
@Override
|
@Override
|
||||||
public Message createMessage(Session session) throws JMSException {
|
public Message createMessage(Session session) throws JMSException {
|
||||||
BytesMessage message = session.createBytesMessage();
|
BytesMessage message = session.createBytesMessage();
|
||||||
message.writeBytes(messageBytes);
|
message.writeBytes(messageBytes);
|
||||||
|
|
||||||
if (flowFileAttributes != null && !flowFileAttributes.isEmpty()) {
|
if (flowFileAttributes != null && !flowFileAttributes.isEmpty()) {
|
||||||
// set message headers and properties
|
// set message headers and properties
|
||||||
for (Entry<String, String> entry : flowFileAttributes.entrySet()) {
|
for (Entry<String, String> entry : flowFileAttributes.entrySet()) {
|
||||||
|
@ -121,18 +96,12 @@ final class JMSPublisher extends JMSWorker {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
private void logUnbuildableDestination(String destinationName, String headerName) {
|
private void logUnbuildableDestination(String destinationName, String headerName) {
|
||||||
this.processLog.warn("Failed to determine destination type from destination name '" + destinationName
|
this.processLog.warn("Failed to determine destination type from destination name '{}'. The '{}' header will not be set.", new Object[] {destinationName, headerName});
|
||||||
+ "'. The '"
|
|
||||||
+ headerName + "' will not be set.");
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
private Destination buildDestination(final String destinationName) {
|
private Destination buildDestination(final String destinationName) {
|
||||||
Destination destination;
|
Destination destination;
|
||||||
if (destinationName.toLowerCase().contains("topic")) {
|
if (destinationName.toLowerCase().contains("topic")) {
|
||||||
|
@ -152,6 +121,7 @@ final class JMSPublisher extends JMSWorker {
|
||||||
} else {
|
} else {
|
||||||
destination = null;
|
destination = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
return destination;
|
return destination;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ import java.nio.channels.Channel;
|
||||||
import javax.jms.Connection;
|
import javax.jms.Connection;
|
||||||
|
|
||||||
import org.apache.nifi.logging.ComponentLog;
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
import org.springframework.jms.connection.CachingConnectionFactory;
|
||||||
import org.springframework.jms.core.JmsTemplate;
|
import org.springframework.jms.core.JmsTemplate;
|
||||||
|
|
||||||
|
|
||||||
|
@ -33,8 +34,9 @@ import org.springframework.jms.core.JmsTemplate;
|
||||||
abstract class JMSWorker {
|
abstract class JMSWorker {
|
||||||
|
|
||||||
protected final JmsTemplate jmsTemplate;
|
protected final JmsTemplate jmsTemplate;
|
||||||
|
|
||||||
protected final ComponentLog processLog;
|
protected final ComponentLog processLog;
|
||||||
|
private final CachingConnectionFactory connectionFactory;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates an instance of this worker initializing it with JMS
|
* Creates an instance of this worker initializing it with JMS
|
||||||
|
@ -44,14 +46,16 @@ abstract class JMSWorker {
|
||||||
* @param jmsTemplate the instance of {@link JmsTemplate}
|
* @param jmsTemplate the instance of {@link JmsTemplate}
|
||||||
* @param processLog the instance of {@link ComponentLog}
|
* @param processLog the instance of {@link ComponentLog}
|
||||||
*/
|
*/
|
||||||
public JMSWorker(JmsTemplate jmsTemplate, ComponentLog processLog) {
|
public JMSWorker(CachingConnectionFactory connectionFactory, JmsTemplate jmsTemplate, ComponentLog processLog) {
|
||||||
|
this.connectionFactory = connectionFactory;
|
||||||
this.jmsTemplate = jmsTemplate;
|
this.jmsTemplate = jmsTemplate;
|
||||||
this.processLog = processLog;
|
this.processLog = processLog;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
public void shutdown() {
|
||||||
*
|
connectionFactory.destroy();
|
||||||
*/
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return this.getClass().getSimpleName() + "[destination:" + this.jmsTemplate.getDefaultDestinationName()
|
return this.getClass().getSimpleName() + "[destination:" + this.jmsTemplate.getDefaultDestinationName()
|
||||||
|
|
|
@ -16,8 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.jms.processors;
|
package org.apache.nifi.jms.processors;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.InputStream;
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -37,8 +35,8 @@ import org.apache.nifi.processor.ProcessSession;
|
||||||
import org.apache.nifi.processor.Processor;
|
import org.apache.nifi.processor.Processor;
|
||||||
import org.apache.nifi.processor.Relationship;
|
import org.apache.nifi.processor.Relationship;
|
||||||
import org.apache.nifi.processor.exception.ProcessException;
|
import org.apache.nifi.processor.exception.ProcessException;
|
||||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
|
||||||
import org.apache.nifi.stream.io.StreamUtils;
|
import org.apache.nifi.stream.io.StreamUtils;
|
||||||
|
import org.springframework.jms.connection.CachingConnectionFactory;
|
||||||
import org.springframework.jms.core.JmsTemplate;
|
import org.springframework.jms.core.JmsTemplate;
|
||||||
import org.springframework.jms.support.JmsHeaders;
|
import org.springframework.jms.support.JmsHeaders;
|
||||||
|
|
||||||
|
@ -91,20 +89,19 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
|
||||||
* Upon success the incoming {@link FlowFile} is transferred to the'success'
|
* Upon success the incoming {@link FlowFile} is transferred to the'success'
|
||||||
* {@link Relationship} and upon failure FlowFile is penalized and
|
* {@link Relationship} and upon failure FlowFile is penalized and
|
||||||
* transferred to the 'failure' {@link Relationship}
|
* transferred to the 'failure' {@link Relationship}
|
||||||
*
|
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected void rendezvousWithJms(ProcessContext context, ProcessSession processSession) throws ProcessException {
|
protected void rendezvousWithJms(ProcessContext context, ProcessSession processSession, JMSPublisher publisher) throws ProcessException {
|
||||||
FlowFile flowFile = processSession.get();
|
FlowFile flowFile = processSession.get();
|
||||||
if (flowFile != null) {
|
if (flowFile != null) {
|
||||||
try {
|
try {
|
||||||
String destinationName = context.getProperty(DESTINATION).evaluateAttributeExpressions(flowFile).getValue();
|
String destinationName = context.getProperty(DESTINATION).evaluateAttributeExpressions(flowFile).getValue();
|
||||||
this.targetResource.publish(destinationName, this.extractMessageBody(flowFile, processSession), flowFile.getAttributes());
|
publisher.publish(destinationName, this.extractMessageBody(flowFile, processSession), flowFile.getAttributes());
|
||||||
processSession.transfer(flowFile, REL_SUCCESS);
|
processSession.transfer(flowFile, REL_SUCCESS);
|
||||||
processSession.getProvenanceReporter().send(flowFile, context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue());
|
processSession.getProvenanceReporter().send(flowFile, context.getProperty(DESTINATION).evaluateAttributeExpressions().getValue());
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
processSession.transfer(flowFile, REL_FAILURE);
|
processSession.transfer(flowFile, REL_FAILURE);
|
||||||
this.getLogger().error("Failed while sending message to JMS via " + this.targetResource, e);
|
this.getLogger().error("Failed while sending message to JMS via " + publisher, e);
|
||||||
context.yield();
|
context.yield();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -122,8 +119,8 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
|
||||||
* Will create an instance of {@link JMSPublisher}
|
* Will create an instance of {@link JMSPublisher}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected JMSPublisher finishBuildingTargetResource(JmsTemplate jmsTemplate, ProcessContext processContext) {
|
protected JMSPublisher finishBuildingJmsWorker(CachingConnectionFactory connectionFactory, JmsTemplate jmsTemplate, ProcessContext processContext) {
|
||||||
return new JMSPublisher(jmsTemplate, this.getLogger());
|
return new JMSPublisher(connectionFactory, jmsTemplate, this.getLogger());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -131,12 +128,7 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
|
||||||
*/
|
*/
|
||||||
private byte[] extractMessageBody(FlowFile flowFile, ProcessSession session) {
|
private byte[] extractMessageBody(FlowFile flowFile, ProcessSession session) {
|
||||||
final byte[] messageContent = new byte[(int) flowFile.getSize()];
|
final byte[] messageContent = new byte[(int) flowFile.getSize()];
|
||||||
session.read(flowFile, new InputStreamCallback() {
|
session.read(flowFile, in -> StreamUtils.fillBuffer(in, messageContent, true));
|
||||||
@Override
|
|
||||||
public void process(final InputStream in) throws IOException {
|
|
||||||
StreamUtils.fillBuffer(in, messageContent, true);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
return messageContent;
|
return messageContent;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -52,13 +52,13 @@ public class CommonTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
static JmsTemplate buildJmsTemplateForDestination(boolean pubSub) {
|
static JmsTemplate buildJmsTemplateForDestination(boolean pubSub) {
|
||||||
ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
|
ConnectionFactory activeMqConnectionFactory = new ActiveMQConnectionFactory("vm://localhost?broker.persistent=false");
|
||||||
"vm://localhost?broker.persistent=false");
|
final ConnectionFactory connectionFactory = new CachingConnectionFactory(activeMqConnectionFactory);
|
||||||
connectionFactory = new CachingConnectionFactory(connectionFactory);
|
|
||||||
|
|
||||||
JmsTemplate jmsTemplate = new JmsTemplate(connectionFactory);
|
JmsTemplate jmsTemplate = new JmsTemplate(connectionFactory);
|
||||||
jmsTemplate.setPubSubDomain(pubSub);
|
jmsTemplate.setPubSubDomain(pubSub);
|
||||||
jmsTemplate.setSessionAcknowledgeMode(Session.CLIENT_ACKNOWLEDGE);
|
jmsTemplate.setSessionAcknowledgeMode(Session.CLIENT_ACKNOWLEDGE);
|
||||||
|
jmsTemplate.setReceiveTimeout(10L);
|
||||||
return jmsTemplate;
|
return jmsTemplate;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,6 +16,10 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.jms.processors;
|
package org.apache.nifi.jms.processors;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -29,45 +33,43 @@ import org.springframework.jms.connection.CachingConnectionFactory;
|
||||||
import org.springframework.jms.core.JmsTemplate;
|
import org.springframework.jms.core.JmsTemplate;
|
||||||
import org.springframework.jms.support.JmsHeaders;
|
import org.springframework.jms.support.JmsHeaders;
|
||||||
|
|
||||||
import static org.junit.Assert.assertNotNull;
|
|
||||||
import static org.mockito.Mockito.mock;
|
|
||||||
import static org.mockito.Mockito.when;
|
|
||||||
|
|
||||||
public class ConsumeJMSTest {
|
public class ConsumeJMSTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void validateSuccessfulConsumeAndTransferToSuccess() throws Exception {
|
public void validateSuccessfulConsumeAndTransferToSuccess() throws Exception {
|
||||||
final String destinationName = "cooQueue";
|
final String destinationName = "cooQueue";
|
||||||
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
JMSPublisher sender = new JMSPublisher(jmsTemplate, mock(ComponentLog.class));
|
try {
|
||||||
final Map<String, String> senderAttributes = new HashMap<>();
|
JMSPublisher sender = new JMSPublisher((CachingConnectionFactory) jmsTemplate.getConnectionFactory(), jmsTemplate, mock(ComponentLog.class));
|
||||||
senderAttributes.put("filename", "message.txt");
|
final Map<String, String> senderAttributes = new HashMap<>();
|
||||||
senderAttributes.put("attribute_from_sender", "some value");
|
senderAttributes.put("filename", "message.txt");
|
||||||
sender.publish(destinationName, "Hey dude!".getBytes(), senderAttributes);
|
senderAttributes.put("attribute_from_sender", "some value");
|
||||||
TestRunner runner = TestRunners.newTestRunner(new ConsumeJMS());
|
sender.publish(destinationName, "Hey dude!".getBytes(), senderAttributes);
|
||||||
JMSConnectionFactoryProviderDefinition cs = mock(JMSConnectionFactoryProviderDefinition.class);
|
TestRunner runner = TestRunners.newTestRunner(new ConsumeJMS());
|
||||||
when(cs.getIdentifier()).thenReturn("cfProvider");
|
JMSConnectionFactoryProviderDefinition cs = mock(JMSConnectionFactoryProviderDefinition.class);
|
||||||
when(cs.getConnectionFactory()).thenReturn(jmsTemplate.getConnectionFactory());
|
when(cs.getIdentifier()).thenReturn("cfProvider");
|
||||||
runner.addControllerService("cfProvider", cs);
|
when(cs.getConnectionFactory()).thenReturn(jmsTemplate.getConnectionFactory());
|
||||||
runner.enableControllerService(cs);
|
runner.addControllerService("cfProvider", cs);
|
||||||
|
runner.enableControllerService(cs);
|
||||||
|
|
||||||
runner.setProperty(PublishJMS.CF_SERVICE, "cfProvider");
|
runner.setProperty(PublishJMS.CF_SERVICE, "cfProvider");
|
||||||
runner.setProperty(ConsumeJMS.DESTINATION, destinationName);
|
runner.setProperty(ConsumeJMS.DESTINATION, destinationName);
|
||||||
runner.setProperty(ConsumeJMS.DESTINATION_TYPE, ConsumeJMS.QUEUE);
|
runner.setProperty(ConsumeJMS.DESTINATION_TYPE, ConsumeJMS.QUEUE);
|
||||||
runner.run(1, false);
|
runner.run(1, false);
|
||||||
//
|
//
|
||||||
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PublishJMS.REL_SUCCESS).get(0);
|
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PublishJMS.REL_SUCCESS).get(0);
|
||||||
assertNotNull(successFF);
|
assertNotNull(successFF);
|
||||||
successFF.assertAttributeExists(JmsHeaders.DESTINATION);
|
successFF.assertAttributeExists(JmsHeaders.DESTINATION);
|
||||||
successFF.assertAttributeEquals(JmsHeaders.DESTINATION, destinationName);
|
successFF.assertAttributeEquals(JmsHeaders.DESTINATION, destinationName);
|
||||||
successFF.assertAttributeExists("filename");
|
successFF.assertAttributeExists("filename");
|
||||||
successFF.assertAttributeEquals("filename", "message.txt");
|
successFF.assertAttributeEquals("filename", "message.txt");
|
||||||
successFF.assertAttributeExists("attribute_from_sender");
|
successFF.assertAttributeExists("attribute_from_sender");
|
||||||
successFF.assertAttributeEquals("attribute_from_sender", "some value");
|
successFF.assertAttributeEquals("attribute_from_sender", "some value");
|
||||||
successFF.assertContentEquals("Hey dude!".getBytes());
|
successFF.assertContentEquals("Hey dude!".getBytes());
|
||||||
String sourceDestination = successFF.getAttribute(ConsumeJMS.JMS_SOURCE_DESTINATION_NAME);
|
String sourceDestination = successFF.getAttribute(ConsumeJMS.JMS_SOURCE_DESTINATION_NAME);
|
||||||
assertNotNull(sourceDestination);
|
assertNotNull(sourceDestination);
|
||||||
|
} finally {
|
||||||
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,9 @@ import static org.mockito.Mockito.mock;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import javax.jms.BytesMessage;
|
import javax.jms.BytesMessage;
|
||||||
import javax.jms.JMSException;
|
import javax.jms.JMSException;
|
||||||
|
@ -48,16 +50,18 @@ public class JMSPublisherConsumerTest {
|
||||||
final String destinationName = "testQueue";
|
final String destinationName = "testQueue";
|
||||||
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
|
|
||||||
JMSPublisher publisher = new JMSPublisher(jmsTemplate, mock(ComponentLog.class));
|
try {
|
||||||
publisher.publish(destinationName, "hellomq".getBytes());
|
JMSPublisher publisher = new JMSPublisher((CachingConnectionFactory) jmsTemplate.getConnectionFactory(), jmsTemplate, mock(ComponentLog.class));
|
||||||
|
publisher.publish(destinationName, "hellomq".getBytes());
|
||||||
|
|
||||||
Message receivedMessage = jmsTemplate.receive(destinationName);
|
Message receivedMessage = jmsTemplate.receive(destinationName);
|
||||||
assertTrue(receivedMessage instanceof BytesMessage);
|
assertTrue(receivedMessage instanceof BytesMessage);
|
||||||
byte[] bytes = new byte[7];
|
byte[] bytes = new byte[7];
|
||||||
((BytesMessage) receivedMessage).readBytes(bytes);
|
((BytesMessage) receivedMessage).readBytes(bytes);
|
||||||
assertEquals("hellomq", new String(bytes));
|
assertEquals("hellomq", new String(bytes));
|
||||||
|
} finally {
|
||||||
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -65,19 +69,22 @@ public class JMSPublisherConsumerTest {
|
||||||
final String destinationName = "testQueue";
|
final String destinationName = "testQueue";
|
||||||
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
|
|
||||||
JMSPublisher publisher = new JMSPublisher(jmsTemplate, mock(ComponentLog.class));
|
try {
|
||||||
Map<String, String> flowFileAttributes = new HashMap<>();
|
JMSPublisher publisher = new JMSPublisher((CachingConnectionFactory) jmsTemplate.getConnectionFactory(), jmsTemplate, mock(ComponentLog.class));
|
||||||
flowFileAttributes.put("foo", "foo");
|
Map<String, String> flowFileAttributes = new HashMap<>();
|
||||||
flowFileAttributes.put(JmsHeaders.REPLY_TO, "myTopic");
|
flowFileAttributes.put("foo", "foo");
|
||||||
publisher.publish(destinationName, "hellomq".getBytes(), flowFileAttributes);
|
flowFileAttributes.put(JmsHeaders.REPLY_TO, "myTopic");
|
||||||
|
publisher.publish(destinationName, "hellomq".getBytes(), flowFileAttributes);
|
||||||
|
|
||||||
Message receivedMessage = jmsTemplate.receive(destinationName);
|
Message receivedMessage = jmsTemplate.receive(destinationName);
|
||||||
assertTrue(receivedMessage instanceof BytesMessage);
|
assertTrue(receivedMessage instanceof BytesMessage);
|
||||||
assertEquals("foo", receivedMessage.getStringProperty("foo"));
|
assertEquals("foo", receivedMessage.getStringProperty("foo"));
|
||||||
assertTrue(receivedMessage.getJMSReplyTo() instanceof Topic);
|
assertTrue(receivedMessage.getJMSReplyTo() instanceof Topic);
|
||||||
assertEquals("myTopic", ((Topic) receivedMessage.getJMSReplyTo()).getTopicName());
|
assertEquals("myTopic", ((Topic) receivedMessage.getJMSReplyTo()).getTopicName());
|
||||||
|
|
||||||
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
} finally {
|
||||||
|
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -91,15 +98,15 @@ public class JMSPublisherConsumerTest {
|
||||||
final String destinationName = "testQueue";
|
final String destinationName = "testQueue";
|
||||||
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
|
|
||||||
jmsTemplate.send(destinationName, new MessageCreator() {
|
|
||||||
@Override
|
|
||||||
public Message createMessage(Session session) throws JMSException {
|
|
||||||
return session.createObjectMessage();
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
JMSConsumer consumer = new JMSConsumer(jmsTemplate, mock(ComponentLog.class));
|
|
||||||
try {
|
try {
|
||||||
|
jmsTemplate.send(destinationName, new MessageCreator() {
|
||||||
|
@Override
|
||||||
|
public Message createMessage(Session session) throws JMSException {
|
||||||
|
return session.createObjectMessage();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
JMSConsumer consumer = new JMSConsumer((CachingConnectionFactory) jmsTemplate.getConnectionFactory(), jmsTemplate, mock(ComponentLog.class));
|
||||||
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void accept(JMSResponse response) {
|
public void accept(JMSResponse response) {
|
||||||
|
@ -116,102 +123,182 @@ public class JMSPublisherConsumerTest {
|
||||||
final String destinationName = "testQueue";
|
final String destinationName = "testQueue";
|
||||||
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
|
|
||||||
jmsTemplate.send(destinationName, new MessageCreator() {
|
try {
|
||||||
@Override
|
jmsTemplate.send(destinationName, new MessageCreator() {
|
||||||
public Message createMessage(Session session) throws JMSException {
|
@Override
|
||||||
TextMessage message = session.createTextMessage("hello from the other side");
|
public Message createMessage(Session session) throws JMSException {
|
||||||
message.setStringProperty("foo", "foo");
|
TextMessage message = session.createTextMessage("hello from the other side");
|
||||||
message.setBooleanProperty("bar", false);
|
message.setStringProperty("foo", "foo");
|
||||||
message.setJMSReplyTo(session.createQueue("fooQueue"));
|
message.setBooleanProperty("bar", false);
|
||||||
return message;
|
message.setJMSReplyTo(session.createQueue("fooQueue"));
|
||||||
}
|
return message;
|
||||||
});
|
}
|
||||||
|
});
|
||||||
|
|
||||||
JMSConsumer consumer = new JMSConsumer(jmsTemplate, mock(ComponentLog.class));
|
JMSConsumer consumer = new JMSConsumer((CachingConnectionFactory) jmsTemplate.getConnectionFactory(), jmsTemplate, mock(ComponentLog.class));
|
||||||
final AtomicBoolean callbackInvoked = new AtomicBoolean();
|
final AtomicBoolean callbackInvoked = new AtomicBoolean();
|
||||||
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void accept(JMSResponse response) {
|
public void accept(JMSResponse response) {
|
||||||
callbackInvoked.set(true);
|
callbackInvoked.set(true);
|
||||||
assertEquals("hello from the other side", new String(response.getMessageBody()));
|
assertEquals("hello from the other side", new String(response.getMessageBody()));
|
||||||
assertEquals("fooQueue", response.getMessageHeaders().get(JmsHeaders.REPLY_TO));
|
assertEquals("fooQueue", response.getMessageHeaders().get(JmsHeaders.REPLY_TO));
|
||||||
assertEquals("foo", response.getMessageProperties().get("foo"));
|
assertEquals("foo", response.getMessageProperties().get("foo"));
|
||||||
assertEquals("false", response.getMessageProperties().get("bar"));
|
assertEquals("false", response.getMessageProperties().get("bar"));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
assertTrue(callbackInvoked.get());
|
assertTrue(callbackInvoked.get());
|
||||||
|
|
||||||
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
} finally {
|
||||||
|
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
|
@Test(timeout = 2000000)
|
||||||
|
public void testMultipleThreads() throws Exception {
|
||||||
|
String destinationName = "testQueue";
|
||||||
|
JmsTemplate publishTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
|
final CountDownLatch consumerTemplateCloseCount = new CountDownLatch(4);
|
||||||
|
|
||||||
|
try {
|
||||||
|
JMSPublisher publisher = new JMSPublisher((CachingConnectionFactory) publishTemplate.getConnectionFactory(), publishTemplate, mock(ComponentLog.class));
|
||||||
|
for (int i = 0; i < 4000; i++) {
|
||||||
|
publisher.publish(destinationName, String.valueOf(i).getBytes(StandardCharsets.UTF_8));
|
||||||
|
}
|
||||||
|
|
||||||
|
final AtomicInteger msgCount = new AtomicInteger(0);
|
||||||
|
|
||||||
|
final ConsumerCallback callback = new ConsumerCallback() {
|
||||||
|
@Override
|
||||||
|
public void accept(JMSResponse response) {
|
||||||
|
msgCount.incrementAndGet();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
final Thread[] threads = new Thread[4];
|
||||||
|
for (int i = 0; i < 4; i++) {
|
||||||
|
final Thread t = new Thread(() -> {
|
||||||
|
JmsTemplate consumeTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
JMSConsumer consumer = new JMSConsumer((CachingConnectionFactory) consumeTemplate.getConnectionFactory(), consumeTemplate, mock(ComponentLog.class));
|
||||||
|
|
||||||
|
for (int j = 0; j < 1000 && msgCount.get() < 4000; j++) {
|
||||||
|
consumer.consume(destinationName, false, false, null, callback);
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
((CachingConnectionFactory) consumeTemplate.getConnectionFactory()).destroy();
|
||||||
|
consumerTemplateCloseCount.countDown();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
threads[i] = t;
|
||||||
|
t.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
int iterations = 0;
|
||||||
|
while (msgCount.get() < 4000) {
|
||||||
|
Thread.sleep(10L);
|
||||||
|
if (++iterations % 100 == 0) {
|
||||||
|
System.out.println(msgCount.get() + " messages received so far");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
((CachingConnectionFactory) publishTemplate.getConnectionFactory()).destroy();
|
||||||
|
|
||||||
|
consumerTemplateCloseCount.await();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test(timeout = 10000)
|
||||||
public void validateMessageRedeliveryWhenNotAcked() throws Exception {
|
public void validateMessageRedeliveryWhenNotAcked() throws Exception {
|
||||||
String destinationName = "testQueue";
|
String destinationName = "testQueue";
|
||||||
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
JmsTemplate jmsTemplate = CommonTest.buildJmsTemplateForDestination(false);
|
||||||
JMSPublisher publisher = new JMSPublisher(jmsTemplate, mock(ComponentLog.class));
|
|
||||||
publisher.publish(destinationName, "1".getBytes(StandardCharsets.UTF_8));
|
|
||||||
publisher.publish(destinationName, "2".getBytes(StandardCharsets.UTF_8));
|
|
||||||
|
|
||||||
JMSConsumer consumer = new JMSConsumer(jmsTemplate, mock(ComponentLog.class));
|
|
||||||
final AtomicBoolean callbackInvoked = new AtomicBoolean();
|
|
||||||
try {
|
try {
|
||||||
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
JMSPublisher publisher = new JMSPublisher((CachingConnectionFactory) jmsTemplate.getConnectionFactory(), jmsTemplate, mock(ComponentLog.class));
|
||||||
@Override
|
publisher.publish(destinationName, "1".getBytes(StandardCharsets.UTF_8));
|
||||||
public void accept(JMSResponse response) {
|
publisher.publish(destinationName, "2".getBytes(StandardCharsets.UTF_8));
|
||||||
callbackInvoked.set(true);
|
|
||||||
assertEquals("1", new String(response.getMessageBody()));
|
|
||||||
throw new RuntimeException("intentional to avoid explicit ack");
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (Exception e) {
|
|
||||||
// ignore
|
|
||||||
}
|
|
||||||
assertTrue(callbackInvoked.get());
|
|
||||||
callbackInvoked.set(false);
|
|
||||||
|
|
||||||
// should receive the same message, but will process it successfully
|
JMSConsumer consumer = new JMSConsumer((CachingConnectionFactory) jmsTemplate.getConnectionFactory(), jmsTemplate, mock(ComponentLog.class));
|
||||||
try {
|
final AtomicBoolean callbackInvoked = new AtomicBoolean();
|
||||||
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
try {
|
||||||
@Override
|
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
||||||
public void accept(JMSResponse response) {
|
@Override
|
||||||
callbackInvoked.set(true);
|
public void accept(JMSResponse response) {
|
||||||
assertEquals("1", new String(response.getMessageBody()));
|
callbackInvoked.set(true);
|
||||||
}
|
assertEquals("1", new String(response.getMessageBody()));
|
||||||
});
|
throw new RuntimeException("intentional to avoid explicit ack");
|
||||||
} catch (Exception e) {
|
}
|
||||||
// ignore
|
});
|
||||||
}
|
} catch (Exception e) {
|
||||||
assertTrue(callbackInvoked.get());
|
// expected
|
||||||
callbackInvoked.set(false);
|
}
|
||||||
|
|
||||||
// receiving next message and fail again
|
assertTrue(callbackInvoked.get());
|
||||||
try {
|
callbackInvoked.set(false);
|
||||||
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
|
||||||
@Override
|
|
||||||
public void accept(JMSResponse response) {
|
|
||||||
callbackInvoked.set(true);
|
|
||||||
assertEquals("2", new String(response.getMessageBody()));
|
|
||||||
throw new RuntimeException("intentional to avoid explicit ack");
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (Exception e) {
|
|
||||||
// ignore
|
|
||||||
}
|
|
||||||
assertTrue(callbackInvoked.get());
|
|
||||||
callbackInvoked.set(false);
|
|
||||||
|
|
||||||
// should receive the same message, but will process it successfully
|
// should receive the same message, but will process it successfully
|
||||||
try {
|
while (!callbackInvoked.get()) {
|
||||||
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void accept(JMSResponse response) {
|
public void accept(JMSResponse response) {
|
||||||
callbackInvoked.set(true);
|
if (response == null) {
|
||||||
assertEquals("2", new String(response.getMessageBody()));
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
callbackInvoked.set(true);
|
||||||
|
assertEquals("1", new String(response.getMessageBody()));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
assertTrue(callbackInvoked.get());
|
||||||
|
callbackInvoked.set(false);
|
||||||
|
|
||||||
|
// receiving next message and fail again
|
||||||
|
try {
|
||||||
|
while (!callbackInvoked.get()) {
|
||||||
|
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
||||||
|
@Override
|
||||||
|
public void accept(JMSResponse response) {
|
||||||
|
if (response == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
callbackInvoked.set(true);
|
||||||
|
assertEquals("2", new String(response.getMessageBody()));
|
||||||
|
throw new RuntimeException("intentional to avoid explicit ack");
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
});
|
} catch (Exception e) {
|
||||||
} catch (Exception e) {
|
// ignore
|
||||||
// ignore
|
}
|
||||||
|
assertTrue(callbackInvoked.get());
|
||||||
|
callbackInvoked.set(false);
|
||||||
|
|
||||||
|
// should receive the same message, but will process it successfully
|
||||||
|
try {
|
||||||
|
while (!callbackInvoked.get()) {
|
||||||
|
consumer.consume(destinationName, false, false, null, new ConsumerCallback() {
|
||||||
|
@Override
|
||||||
|
public void accept(JMSResponse response) {
|
||||||
|
if (response == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
callbackInvoked.set(true);
|
||||||
|
assertEquals("2", new String(response.getMessageBody()));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
// ignore
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
||||||
}
|
}
|
||||||
((CachingConnectionFactory) jmsTemplate.getConnectionFactory()).destroy();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,7 +39,7 @@ import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
public class PublishJMSTest {
|
public class PublishJMSTest {
|
||||||
|
|
||||||
@Test
|
@Test(timeout = 10000)
|
||||||
public void validateSuccessfulPublishAndTransferToSuccess() throws Exception {
|
public void validateSuccessfulPublishAndTransferToSuccess() throws Exception {
|
||||||
ActiveMQConnectionFactory cf = new ActiveMQConnectionFactory("vm://localhost?broker.persistent=false");
|
ActiveMQConnectionFactory cf = new ActiveMQConnectionFactory("vm://localhost?broker.persistent=false");
|
||||||
|
|
||||||
|
@ -60,7 +60,7 @@ public class PublishJMSTest {
|
||||||
attributes.put("foo", "foo");
|
attributes.put("foo", "foo");
|
||||||
attributes.put(JmsHeaders.REPLY_TO, "cooQueue");
|
attributes.put(JmsHeaders.REPLY_TO, "cooQueue");
|
||||||
runner.enqueue("Hey dude!".getBytes(), attributes);
|
runner.enqueue("Hey dude!".getBytes(), attributes);
|
||||||
runner.run(1, false);
|
runner.run(1, false); // Run once but don't shut down because we want the Connection Factory left in tact so that we can use it.
|
||||||
|
|
||||||
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PublishJMS.REL_SUCCESS).get(0);
|
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PublishJMS.REL_SUCCESS).get(0);
|
||||||
assertNotNull(successFF);
|
assertNotNull(successFF);
|
||||||
|
@ -72,6 +72,8 @@ public class PublishJMSTest {
|
||||||
assertEquals("Hey dude!", new String(messageBytes));
|
assertEquals("Hey dude!", new String(messageBytes));
|
||||||
assertEquals("cooQueue", ((Queue) message.getJMSReplyTo()).getQueueName());
|
assertEquals("cooQueue", ((Queue) message.getJMSReplyTo()).getQueueName());
|
||||||
assertEquals("foo", message.getStringProperty("foo"));
|
assertEquals("foo", message.getStringProperty("foo"));
|
||||||
|
|
||||||
|
runner.run(1, true); // Run once just so that we can trigger the shutdown of the Connection Factory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -96,7 +98,7 @@ public class PublishJMSTest {
|
||||||
attributes.put("foo", "foo");
|
attributes.put("foo", "foo");
|
||||||
attributes.put(JmsHeaders.REPLY_TO, "cooQueue");
|
attributes.put(JmsHeaders.REPLY_TO, "cooQueue");
|
||||||
runner.enqueue("Hey dude!".getBytes(), attributes);
|
runner.enqueue("Hey dude!".getBytes(), attributes);
|
||||||
runner.run(1, false);
|
runner.run(1, false); // Run once but don't shut down because we want the Connection Factory left in tact so that we can use it.
|
||||||
|
|
||||||
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PublishJMS.REL_SUCCESS).get(0);
|
final MockFlowFile successFF = runner.getFlowFilesForRelationship(PublishJMS.REL_SUCCESS).get(0);
|
||||||
assertNotNull(successFF);
|
assertNotNull(successFF);
|
||||||
|
@ -108,6 +110,8 @@ public class PublishJMSTest {
|
||||||
assertEquals("Hey dude!", new String(messageBytes));
|
assertEquals("Hey dude!", new String(messageBytes));
|
||||||
assertEquals("cooQueue", ((Queue) message.getJMSReplyTo()).getQueueName());
|
assertEquals("cooQueue", ((Queue) message.getJMSReplyTo()).getQueueName());
|
||||||
assertEquals("foo", message.getStringProperty("foo"));
|
assertEquals("foo", message.getStringProperty("foo"));
|
||||||
|
|
||||||
|
runner.run(1, true); // Run once just so that we can trigger the shutdown of the Connection Factory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
Loading…
Reference in New Issue