From 14b3349e8d5bab515d83b854f23fa00439bd408d Mon Sep 17 00:00:00 2001 From: Mike Moser Date: Fri, 4 Dec 2015 14:26:14 -0500 Subject: [PATCH] NIFI-1254 remove Destination Type property from GetJMSQueue and GetJMSTopic Signed-off-by: Mark Payne --- .../nifi/processors/standard/JmsConsumer.java | 2 - .../processors/standard/TestGetJMSQueue.java | 69 ++++++++++++------- 2 files changed, 45 insertions(+), 26 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java index d4e1969712..ea70d52d71 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/JmsConsumer.java @@ -21,7 +21,6 @@ import static org.apache.nifi.processors.standard.util.JmsProperties.ACK_MODE_CL import static org.apache.nifi.processors.standard.util.JmsProperties.BATCH_SIZE; import static org.apache.nifi.processors.standard.util.JmsProperties.CLIENT_ID_PREFIX; import static org.apache.nifi.processors.standard.util.JmsProperties.DESTINATION_NAME; -import static org.apache.nifi.processors.standard.util.JmsProperties.DESTINATION_TYPE; import static org.apache.nifi.processors.standard.util.JmsProperties.JMS_PROPS_TO_ATTRIBUTES; import static org.apache.nifi.processors.standard.util.JmsProperties.JMS_PROVIDER; import static org.apache.nifi.processors.standard.util.JmsProperties.MESSAGE_SELECTOR; @@ -90,7 +89,6 @@ public abstract class JmsConsumer extends AbstractProcessor { descriptors.add(USERNAME); descriptors.add(PASSWORD); descriptors.add(SSL_CONTEXT_SERVICE); - descriptors.add(DESTINATION_TYPE); descriptors.add(ACKNOWLEDGEMENT_MODE); descriptors.add(MESSAGE_SELECTOR); descriptors.add(JMS_PROPS_TO_ATTRIBUTES); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetJMSQueue.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetJMSQueue.java index bfc56a5ed1..3d1447d054 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetJMSQueue.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetJMSQueue.java @@ -42,15 +42,15 @@ public class TestGetJMSQueue { @Test public void testSendTextToQueue() throws Exception { - GetJMSQueue getJmsQueue = new GetJMSQueue(); - TestRunner runner = TestRunners.newTestRunner(getJmsQueue); - runner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); - runner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); - runner.setProperty(JmsProperties.DESTINATION_TYPE, JmsProperties.DESTINATION_TYPE_QUEUE); - runner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); - runner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); + PutJMS putJms = new PutJMS(); + TestRunner putRunner = TestRunners.newTestRunner(putJms); + putRunner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); + putRunner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); + putRunner.setProperty(JmsProperties.DESTINATION_TYPE, JmsProperties.DESTINATION_TYPE_QUEUE); + putRunner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); + putRunner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); - WrappedMessageProducer wrappedProducer = JmsFactory.createMessageProducer(runner.getProcessContext(), true); + WrappedMessageProducer wrappedProducer = JmsFactory.createMessageProducer(putRunner.getProcessContext(), true); final Session jmsSession = wrappedProducer.getSession(); final MessageProducer producer = wrappedProducer.getProducer(); final Message message = jmsSession.createTextMessage("Hello World"); @@ -58,6 +58,13 @@ public class TestGetJMSQueue { producer.send(message); jmsSession.commit(); + GetJMSQueue getJmsQueue = new GetJMSQueue(); + TestRunner runner = TestRunners.newTestRunner(getJmsQueue); + runner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); + runner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); + runner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); + runner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); + runner.run(); List flowFiles = runner @@ -73,14 +80,14 @@ public class TestGetJMSQueue { @Test public void testSendBytesToQueue() throws Exception { - GetJMSQueue getJmsQueue = new GetJMSQueue(); - TestRunner runner = TestRunners.newTestRunner(getJmsQueue); - runner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); - runner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); - runner.setProperty(JmsProperties.DESTINATION_TYPE, JmsProperties.DESTINATION_TYPE_QUEUE); - runner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); - runner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); - WrappedMessageProducer wrappedProducer = JmsFactory.createMessageProducer(runner.getProcessContext(), true); + PutJMS putJms = new PutJMS(); + TestRunner putRunner = TestRunners.newTestRunner(putJms); + putRunner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); + putRunner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); + putRunner.setProperty(JmsProperties.DESTINATION_TYPE, JmsProperties.DESTINATION_TYPE_QUEUE); + putRunner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); + putRunner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); + WrappedMessageProducer wrappedProducer = JmsFactory.createMessageProducer(putRunner.getProcessContext(), true); final Session jmsSession = wrappedProducer.getSession(); final MessageProducer producer = wrappedProducer.getProducer(); final BytesMessage message = jmsSession.createBytesMessage(); @@ -89,6 +96,13 @@ public class TestGetJMSQueue { producer.send(message); jmsSession.commit(); + GetJMSQueue getJmsQueue = new GetJMSQueue(); + TestRunner runner = TestRunners.newTestRunner(getJmsQueue); + runner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); + runner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); + runner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); + runner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); + runner.run(); List flowFiles = runner @@ -104,14 +118,14 @@ public class TestGetJMSQueue { @Test public void testSendStreamToQueue() throws Exception { - GetJMSQueue getJmsQueue = new GetJMSQueue(); - TestRunner runner = TestRunners.newTestRunner(getJmsQueue); - runner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); - runner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); - runner.setProperty(JmsProperties.DESTINATION_TYPE, JmsProperties.DESTINATION_TYPE_QUEUE); - runner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); - runner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); - WrappedMessageProducer wrappedProducer = JmsFactory.createMessageProducer(runner.getProcessContext(), true); + PutJMS putJms = new PutJMS(); + TestRunner putRunner = TestRunners.newTestRunner(putJms); + putRunner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); + putRunner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); + putRunner.setProperty(JmsProperties.DESTINATION_TYPE, JmsProperties.DESTINATION_TYPE_QUEUE); + putRunner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); + putRunner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); + WrappedMessageProducer wrappedProducer = JmsFactory.createMessageProducer(putRunner.getProcessContext(), true); final Session jmsSession = wrappedProducer.getSession(); final MessageProducer producer = wrappedProducer.getProducer(); @@ -121,6 +135,13 @@ public class TestGetJMSQueue { producer.send(message); jmsSession.commit(); + GetJMSQueue getJmsQueue = new GetJMSQueue(); + TestRunner runner = TestRunners.newTestRunner(getJmsQueue); + runner.setProperty(JmsProperties.JMS_PROVIDER, JmsProperties.ACTIVEMQ_PROVIDER); + runner.setProperty(JmsProperties.URL, "vm://localhost?broker.persistent=false"); + runner.setProperty(JmsProperties.DESTINATION_NAME, "queue.testing"); + runner.setProperty(JmsProperties.ACKNOWLEDGEMENT_MODE, JmsProperties.ACK_MODE_AUTO); + runner.run(); List flowFiles = runner