@Override public final void initialize(final ProcessorInitializationContext context) { identifier = context.getIdentifier(); logger = context.getLogger(); serviceLookup = context.getControllerServiceLookup(); nodeTypeProvider = context.getNodeTypeProvider(); init(context); description = getClass().getSimpleName() + "[id=" + identifier + "]"; }
/** * Test MapMessage to FlowFile conversion * * @throws java.lang.Exception ex */ @Test public void testMap2FlowFileMapMessage() throws Exception { TestRunner runner = TestRunners.newTestRunner(GetJMSQueue.class); MapMessage mapMessage = createMapMessage(); ProcessContext context = runner.getProcessContext(); ProcessSession session = runner.getProcessSessionFactory().createSession(); ProcessorInitializationContext pic = new MockProcessorInitializationContext(runner.getProcessor(), (MockProcessContext) runner.getProcessContext()); JmsProcessingSummary summary = JmsConsumer.map2FlowFile(context, session, mapMessage, true, pic.getLogger()); assertEquals("MapMessage should not create FlowFile content", 0, summary.getBytesReceived()); Map<String, String> attributes = summary.getLastFlowFile().getAttributes(); assertEquals("", "Arnold", attributes.get(JmsConsumer.MAP_MESSAGE_PREFIX + "name")); assertEquals("", "97", attributes.get(JmsConsumer.MAP_MESSAGE_PREFIX + "age")); assertEquals("", "89686.564", attributes.get(JmsConsumer.MAP_MESSAGE_PREFIX + "xyz")); assertEquals("", "true", attributes.get(JmsConsumer.MAP_MESSAGE_PREFIX + "good")); }
@Test public void testMap2FlowFileTextMessage() throws Exception { TestRunner runner = TestRunners.newTestRunner(GetJMSQueue.class); TextMessage textMessage = new ActiveMQTextMessage(); String payload = "Hello world!"; textMessage.setText(payload); ProcessContext context = runner.getProcessContext(); ProcessSession session = runner.getProcessSessionFactory().createSession(); ProcessorInitializationContext pic = new MockProcessorInitializationContext(runner.getProcessor(), (MockProcessContext) runner.getProcessContext()); JmsProcessingSummary summary = JmsConsumer.map2FlowFile(context, session, textMessage, true, pic.getLogger()); assertEquals("TextMessage content length should equal to FlowFile content size", payload.length(), summary.getLastFlowFile().getSize()); final byte[] buffer = new byte[payload.length()]; runner.clearTransferState(); session.read(summary.getLastFlowFile(), new InputStreamCallback() { @Override public void process(InputStream in) throws IOException { StreamUtils.fillBuffer(in, buffer, false); } }); String contentString = new String(buffer, "UTF-8"); assertEquals("", payload, contentString); }
/** * Test BytesMessage to FlowFile conversion * * @throws java.lang.Exception ex */ @Test public void testMap2FlowFileBytesMessage() throws Exception { TestRunner runner = TestRunners.newTestRunner(GetJMSQueue.class); BytesMessage bytesMessage = new ActiveMQBytesMessage(); String sourceString = "Apache NiFi is an easy to use, powerful, and reliable system to process and distribute data.!"; byte[] payload = sourceString.getBytes("UTF-8"); bytesMessage.writeBytes(payload); bytesMessage.reset(); ProcessContext context = runner.getProcessContext(); ProcessSession session = runner.getProcessSessionFactory().createSession(); ProcessorInitializationContext pic = new MockProcessorInitializationContext(runner.getProcessor(), (MockProcessContext) runner.getProcessContext()); JmsProcessingSummary summary = JmsConsumer.map2FlowFile(context, session, bytesMessage, true, pic.getLogger()); assertEquals("BytesMessage content length should equal to FlowFile content size", payload.length, summary.getLastFlowFile().getSize()); final byte[] buffer = new byte[payload.length]; runner.clearTransferState(); session.read(summary.getLastFlowFile(), new InputStreamCallback() { @Override public void process(InputStream in) throws IOException { StreamUtils.fillBuffer(in, buffer, false); } }); String contentString = new String(buffer, "UTF-8"); assertEquals("", sourceString, contentString); }