NIFI-856 - Implements ListenLumberjack Processor This closes #290.
Signed-off-by: Aldrin Piri <ald...@apache.org> Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/5a897915 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/5a897915 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/5a897915 Branch: refs/heads/master Commit: 5a8979150ccef1bee9f90270dd1e37efa1d020ea Parents: 54c5c28 Author: Andre F de Miranda <trix...@users.noreply.github.com> Authored: Sat Mar 19 01:41:35 2016 +1100 Committer: Aldrin Piri <ald...@apache.org> Committed: Thu Jun 2 21:58:41 2016 -0400 ---------------------------------------------------------------------- nifi-assembly/pom.xml | 5 + .../nifi-lumberjack-nar/pom.xml | 42 +++ .../src/main/resources/META-INF/LICENSE | 202 +++++++++++++ .../src/main/resources/META-INF/NOTICE | 5 + .../nifi-lumberjack-processors/pom.xml | 76 +++++ .../processors/lumberjack/ListenLumberjack.java | 232 ++++++++++++++ .../lumberjack/event/LumberjackEvent.java | 46 +++ .../event/LumberjackEventFactory.java | 38 +++ .../lumberjack/event/LumberjackMetadata.java | 26 ++ .../lumberjack/frame/LumberjackDecoder.java | 299 +++++++++++++++++++ .../lumberjack/frame/LumberjackEncoder.java | 47 +++ .../lumberjack/frame/LumberjackFrame.java | 115 +++++++ .../frame/LumberjackFrameException.java | 32 ++ .../lumberjack/frame/LumberjackState.java | 28 ++ .../handler/LumberjackFrameHandler.java | 114 +++++++ .../LumberjackSSLSocketChannelHandler.java | 94 ++++++ .../handler/LumberjackSocketChannelHandler.java | 103 +++++++ .../LumberjackSocketChannelHandlerFactory.java | 56 ++++ .../response/LumberjackChannelResponse.java | 42 +++ .../lumberjack/response/LumberjackResponse.java | 62 ++++ .../org.apache.nifi.processor.Processor | 15 + .../event/TestLumberjackEventFactory.java | 54 ++++ .../lumberjack/frame/TestLumberjackDecoder.java | 101 +++++++ .../lumberjack/frame/TestLumberjackEncoder.java | 49 +++ .../lumberjack/frame/TestLumberjackFrame.java | 38 +++ .../handler/TestLumberjackFrameHandler.java | 157 ++++++++++ .../TestLumberjackSocketChannelHandler.java | 207 +++++++++++++ nifi-nar-bundles/nifi-lumberjack-bundle/pom.xml | 34 +++ nifi-nar-bundles/pom.xml | 1 + pom.xml | 12 +- 30 files changed, 2329 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-assembly/pom.xml ---------------------------------------------------------------------- diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index aad2561..d226726 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -284,6 +284,11 @@ language governing permissions and limitations under the License. --> </dependency> <dependency> <groupId>org.apache.nifi</groupId> + <artifactId>nifi-lumberjack-nar</artifactId> + <type>nar</type> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> <artifactId>nifi-amqp-nar</artifactId> <type>nar</type> </dependency> http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml new file mode 100644 index 0000000..747f4d9 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/pom.xml @@ -0,0 +1,42 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <parent> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-lumberjack-bundle</artifactId> + <version>1.0.0-SNAPSHOT</version> + </parent> + + <artifactId>nifi-lumberjack-nar</artifactId> + <version>1.0.0-SNAPSHOT</version> + <packaging>nar</packaging> + + <dependencies> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-lumberjack-processors</artifactId> + <version>1.0.0-SNAPSHOT</version> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-standard-services-api-nar</artifactId> + <type>nar</type> + </dependency> + </dependencies> + +</project> http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/LICENSE ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 0000000..489ee58 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/NOTICE ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000..7e0e4a4 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-nar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,5 @@ +nifi-lumberjack-nar +Copyright 2014-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml new file mode 100644 index 0000000..7709a99 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/pom.xml @@ -0,0 +1,76 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <parent> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-lumberjack-bundle</artifactId> + <version>1.0.0-SNAPSHOT</version> + </parent> + + <artifactId>nifi-lumberjack-processors</artifactId> + <packaging>jar</packaging> + + <dependencies> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-api</artifactId> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-processor-utils</artifactId> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-mock</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-socket-utils</artifactId> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-utils</artifactId> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-flowfile-packager</artifactId> + </dependency> + <dependency> + <groupId>org.apache.nifi</groupId> + <artifactId>nifi-ssl-context-service-api</artifactId> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-simple</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>junit</groupId> + <artifactId>junit</artifactId> + <version>4.11</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>com.google.code.gson</groupId> + <artifactId>gson</artifactId> + <version>2.2.4</version> + </dependency> + </dependencies> +</project> http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java new file mode 100644 index 0000000..2716cfe --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.flowfile.attributes.FlowFileAttributeKey; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.util.listen.AbstractListenEventBatchingProcessor; +import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher; +import org.apache.nifi.processor.util.listen.dispatcher.ChannelDispatcher; +import org.apache.nifi.processor.util.listen.dispatcher.SocketChannelDispatcher; +import org.apache.nifi.processor.util.listen.event.EventFactory; +import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory; +import org.apache.nifi.processor.util.listen.response.ChannelResponder; +import org.apache.nifi.processor.util.listen.response.ChannelResponse; +import org.apache.nifi.processors.lumberjack.event.LumberjackEvent; +import org.apache.nifi.processors.lumberjack.event.LumberjackEventFactory; +import org.apache.nifi.processors.lumberjack.frame.LumberjackEncoder; +import org.apache.nifi.processors.lumberjack.handler.LumberjackSocketChannelHandlerFactory; +import org.apache.nifi.processors.lumberjack.response.LumberjackChannelResponse; +import org.apache.nifi.processors.lumberjack.response.LumberjackResponse; +import org.apache.nifi.ssl.SSLContextService; + +import com.google.gson.Gson; + +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +@Tags({"listen", "lumberjack", "tcp", "logs"}) +@CapabilityDescription("Listens for Lumberjack messages being sent to a given port over TCP. Each message will be " + + "acknowledged after successfully writing the message to a FlowFile. Each FlowFile will contain data " + + "portion of one or more Lumberjack frames. In the case where the Lumberjack frames contain syslog messages, the " + + "output of this processor can be sent to a ParseSyslog processor for further processing.") +@WritesAttributes({ + @WritesAttribute(attribute = "lumberjack.sender", description = "The sending host of the messages."), + @WritesAttribute(attribute = "lumberjack.port", description = "The sending port the messages were received over."), + @WritesAttribute(attribute = "lumberjack.sequencenumber", description = "The sequence number of the message. Only included if <Batch Size> is 1."), + @WritesAttribute(attribute = "lumberjack.*", description = "The keys and respective values as sent by the lumberjack producer. Only included if <Batch Size> is 1."), + @WritesAttribute(attribute = "mime.type", description = "The mime.type of the content which is text/plain") +}) +@SeeAlso(classNames = {"org.apache.nifi.processors.standard.ParseSyslog"}) +public class ListenLumberjack extends AbstractListenEventBatchingProcessor<LumberjackEvent> { + + public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("SSL Context Service") + .description("The Controller Service to use in order to obtain an SSL Context. If this property is set, " + + "messages will be received over a secure connection. Note that as Lumberjack client requires" + + "two-way SSL authentication, the controller MUST have a truststore and a keystore to work" + + "properly.") + .required(true) + .identifiesControllerService(SSLContextService.class) + .build(); + + @Override + protected List<PropertyDescriptor> getAdditionalProperties() { + return Arrays.asList( + MAX_CONNECTIONS, + SSL_CONTEXT_SERVICE + ); + } + + @Override + protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) { + final List<ValidationResult> results = new ArrayList<>(); + + final SSLContextService sslContextService = validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + + if (sslContextService != null && sslContextService.isTrustStoreConfigured() == false) { + results.add(new ValidationResult.Builder() + .explanation("The context service must have a truststore configured for the lumberjack forwarder client to work correctly") + .valid(false).subject(SSL_CONTEXT_SERVICE.getName()).build()); + } + + return results; + } + + private volatile LumberjackEncoder lumberjackEncoder; + + + @Override + @OnScheduled + public void onScheduled(ProcessContext context) throws IOException { + super.onScheduled(context); + // wanted to ensure charset was already populated here + lumberjackEncoder = new LumberjackEncoder(); + } + + @Override + protected ChannelDispatcher createDispatcher(final ProcessContext context, final BlockingQueue<LumberjackEvent> events) throws IOException { + final EventFactory<LumberjackEvent> eventFactory = new LumberjackEventFactory(); + final ChannelHandlerFactory<LumberjackEvent, AsyncChannelDispatcher> handlerFactory = new LumberjackSocketChannelHandlerFactory<>(); + + final int maxConnections = context.getProperty(MAX_CONNECTIONS).asInteger(); + final int bufferSize = context.getProperty(RECV_BUFFER_SIZE).asDataSize(DataUnit.B).intValue(); + final Charset charSet = Charset.forName(context.getProperty(CHARSET).getValue()); + + // initialize the buffer pool based on max number of connections and the buffer size + final BlockingQueue<ByteBuffer> bufferPool = createBufferPool(maxConnections, bufferSize); + + // if an SSLContextService was provided then create an SSLContext to pass down to the dispatcher + SSLContext sslContext = null; + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + if (sslContextService != null) { + sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED); + } + + // if we decide to support SSL then get the context and pass it in here + return new SocketChannelDispatcher<>(eventFactory, handlerFactory, bufferPool, events, + getLogger(), maxConnections, sslContext, charSet); + } + + + @Override + protected String getBatchKey(LumberjackEvent event) { + return event.getSender(); + } + + protected void respond(final LumberjackEvent event, final LumberjackResponse lumberjackResponse) { + final ChannelResponse response = new LumberjackChannelResponse(lumberjackEncoder, lumberjackResponse); + + final ChannelResponder responder = event.getResponder(); + responder.addResponse(response); + try { + responder.respond(); + } catch (IOException e) { + getLogger().error("Error sending response for transaction {} due to {}", + new Object[]{event.getSeqNumber(), e.getMessage()}, e); + } + } + + protected void postProcess(final ProcessContext context, final ProcessSession session, final List<LumberjackEvent> events) { + // first commit the session so we guarantee we have all the events successfully + // written to FlowFiles and transferred to the success relationship + session.commit(); + // respond to each event to acknowledge successful receipt + for (final LumberjackEvent event : events) { + respond(event, LumberjackResponse.ok(event.getSeqNumber())); + } + } + + @Override + protected String getTransitUri(FlowFileEventBatch batch) { + final String sender = batch.getEvents().get(0).getSender(); + final String senderHost = sender.startsWith("/") && sender.length() > 1 ? sender.substring(1) : sender; + final String transitUri = new StringBuilder().append("lumberjack").append("://").append(senderHost).append(":") + .append(port).toString(); + return transitUri; + } + + @Override + protected Map<String, String> getAttributes(FlowFileEventBatch batch) { + final List<LumberjackEvent> events = batch.getEvents(); + // the sender and command will be the same for all events based on the batch key + final String sender = events.get(0).getSender(); + final int numAttributes = events.size() == 1 ? 5 : 4; + final Map<String, String> attributes = new HashMap<>(numAttributes); + attributes.put(LumberjackAttributes.SENDER.key(), sender); + attributes.put(LumberjackAttributes.PORT.key(), String.valueOf(port)); + attributes.put(CoreAttributes.MIME_TYPE.key(), "text/plain"); + // if there was only one event then we can pass on the transaction + // NOTE: we could pass on all the transaction ids joined together + if (events.size() == 1) { + attributes.put(LumberjackAttributes.SEQNUMBER.key(), String.valueOf(events.get(0).getSeqNumber())); + + // Convert the serialized fields from JSON + String serialFields = String.valueOf(events.get(0).getFields()); + Gson jsonObject = new Gson(); + + Map<String, String> fields = jsonObject.fromJson(serialFields, Map.class); + + for (Map.Entry<String, String> entry : fields.entrySet()) { + attributes.put(LumberjackAttributes.FIELDS.key().concat(".").concat(entry.getKey()), entry.getValue()); + } + } + return attributes; + } + + public enum LumberjackAttributes implements FlowFileAttributeKey { + SENDER("lumberjack.sender"), + PORT("lumberjack.port"), + SEQNUMBER("lumberjack.sequencenumber"), + FIELDS("lumberjack.fields"); + + private final String key; + + LumberjackAttributes(String key) { + this.key = key; + } + + @Override + public String key() { + return key; + } + } +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEvent.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEvent.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEvent.java new file mode 100644 index 0000000..c4973bc --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEvent.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.event; + +import org.apache.nifi.processor.util.listen.event.StandardEvent; +import org.apache.nifi.processor.util.listen.response.ChannelResponder; + +import java.nio.channels.SocketChannel; + +/** + * A Lumberjack event which adds the transaction number and command to the StandardEvent. + */ +public class LumberjackEvent extends StandardEvent<SocketChannel> { + + private final long seqNumber; + private final String fields; + + public LumberjackEvent(final String sender, final byte[] data, final ChannelResponder<SocketChannel> responder, final long seqNumber, String fields) { + super(sender, data, responder); + this.seqNumber = seqNumber; + this.fields = fields; + } + + public long getSeqNumber() { + return seqNumber; + } + + public String getFields() { + return fields; + } + +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEventFactory.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEventFactory.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEventFactory.java new file mode 100644 index 0000000..cdb2be2 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackEventFactory.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.event; + +import org.apache.nifi.processor.util.listen.event.EventFactory; +import org.apache.nifi.processor.util.listen.response.ChannelResponder; + +import java.util.Map; + +/** + * An EventFactory implementation to create LumberjackEvents. + */ +public class LumberjackEventFactory implements EventFactory<LumberjackEvent> { + + @Override + public LumberjackEvent create(final byte[] data, final Map<String, String> metadata, final ChannelResponder responder) { + final String sender = metadata.get(EventFactory.SENDER_KEY); + final long seqNumber = Long.valueOf(metadata.get(LumberjackMetadata.SEQNUMBER_KEY)); + final String fields = metadata.get(LumberjackMetadata.FIELDS_KEY); + + return new LumberjackEvent(sender, data, responder, seqNumber, fields); + } + +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackMetadata.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackMetadata.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackMetadata.java new file mode 100644 index 0000000..1233c27 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/event/LumberjackMetadata.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.event; + +/** + * Metadata keys for Lumberjack. + */ +public interface LumberjackMetadata { + + String SEQNUMBER_KEY = "lumberjack.sequencenumber"; + String FIELDS_KEY = "lumberjack.fields"; +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackDecoder.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackDecoder.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackDecoder.java new file mode 100644 index 0000000..eec9022 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackDecoder.java @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.frame; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.util.LinkedList; +import java.util.List; +import java.util.zip.InflaterInputStream; + +import org.apache.nifi.stream.io.ByteArrayInputStream; +import org.apache.nifi.stream.io.ByteArrayOutputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Decodes a Lumberjack frame by maintaining a state based on each byte that has been processed. This class + * should not be shared by multiple threads. + */ +public class LumberjackDecoder { + + static final Logger logger = LoggerFactory.getLogger(LumberjackDecoder.class); + + private LumberjackFrame.Builder frameBuilder; + private LumberjackState currState = LumberjackState.VERSION; + private byte decodedFrameType; + + private byte[] decompressedData; + + private final Charset charset; + private final ByteArrayOutputStream currBytes; + + private long windowSize; + + public static final byte FRAME_WINDOWSIZE = 0x57, FRAME_DATA = 0x44, FRAME_COMPRESSED = 0x43, FRAME_ACK = 0x41, FRAME_JSON = 0x4a; + + /** + * @param charset the charset to decode bytes from the Lumberjack frame + */ + public LumberjackDecoder(final Charset charset) { + this(charset, new ByteArrayOutputStream(4096)); + } + + /** + * @param charset the charset to decode bytes from the Lumberjack frame + * @param buffer a buffer to use while processing the bytes + */ + public LumberjackDecoder(final Charset charset, final ByteArrayOutputStream buffer) { + this.charset = charset; + this.currBytes = buffer; + this.frameBuilder = new LumberjackFrame.Builder(); + this.decodedFrameType = 0x00; + } + + /** + * Resets this decoder back to its initial state. + */ + public void reset() { + frameBuilder = new LumberjackFrame.Builder(); + currState = LumberjackState.VERSION; + decodedFrameType = 0x00; + currBytes.reset(); + } + + /** + * Process the next byte from the channel, updating the builder and state accordingly. + * + * @param currByte the next byte to process + * @preturn true if a frame is ready to be retrieved, false otherwise + */ + public boolean process(final byte currByte) throws LumberjackFrameException { + try { + switch (currState) { + case VERSION: + processVERSION(currByte); + break; + case FRAMETYPE: + processFRAMETYPE(currByte); + break; + case PAYLOAD: + processPAYLOAD(currByte); + if (frameBuilder.frameType == FRAME_WINDOWSIZE && currState == LumberjackState.COMPLETE) { + return true; + } else if (frameBuilder.frameType == FRAME_COMPRESSED && currState == LumberjackState.COMPLETE) { + return true; + } else { + break; + } + case COMPLETE: + return true; + default: + break; + } + return false; + } catch (Exception e) { + throw new LumberjackFrameException("Error decoding Lumberjack frame: " + e.getMessage(), e); + } + } + + + /** + * Returns the decoded frame and resets the decoder for the next frame. + * This method should be called after checking isComplete(). + * + * @return the LumberjackFrame that was decoded + */ + public List<LumberjackFrame> getFrames() throws LumberjackFrameException { + List<LumberjackFrame> frames = new LinkedList<>(); + + if (currState != LumberjackState.COMPLETE) { + throw new LumberjackFrameException("Must be at the trailer of a frame"); + } + try { + if (currState == LumberjackState.COMPLETE && frameBuilder.frameType == FRAME_COMPRESSED) { + logger.debug("Frame is compressed, will iterate to decode", new Object[]{}); + // LumberjackDecoder decompressedDecoder = new LumberjackDecoder(); + + // Zero currBytes, currState and frameBuilder prior to iteration over + // decompressed bytes + currBytes.reset(); + frameBuilder.reset(); + currState = LumberjackState.VERSION; + + // Run over decompressed data. + frames = processDECOMPRESSED(decompressedData); + + } else { + final LumberjackFrame frame = frameBuilder.build(); + currBytes.reset(); + frameBuilder.reset(); + currState = LumberjackState.VERSION; + frames.add(frame); + } + return frames; + + } catch (Exception e) { + throw new LumberjackFrameException("Error decoding Lumberjack frame: " + e.getMessage(), e); + } + } + + private List<LumberjackFrame> processDECOMPRESSED(byte[] decompressedData) { + List<LumberjackFrame> frames = new LinkedList<>(); + LumberjackFrame.Builder internalFrameBuilder = new LumberjackFrame.Builder(); + ByteBuffer currentData = ByteBuffer.wrap(decompressedData); + + // Lumberjack has a weird approach to frames, where compressed frames embed D(ata) or J(SON) frames. + // inside a compressed input. + // Or astated in the documentation: + // + // "As an example, you could have 3 data frames compressed into a single + // 'compressed' frame type: 1D{k,v}{k,v}1D{k,v}{k,v}1D{k,v}{k,v}" + // + // Therefore, instead of calling process method again, just iterate over each of + // the frames and split them so they can be processed by LumberjackFrameHandler + + while (currentData.hasRemaining()) { + + int payloadLength = 0; + + internalFrameBuilder.version = currentData.get(); + internalFrameBuilder.frameType = currentData.get(); + internalFrameBuilder.seqNumber = currentData.getInt() & 0x00000000ffffffffL; + currentData.mark(); + + // Set the payloadLength to negative to avoid doing math + // around valueLength and valueLength + payloadLength = payloadLength - currentData.position(); + + long pairCount = currentData.getInt() & 0x00000000ffffffffL; + for (int i = 0; i < pairCount; i++) { + long keyLength = currentData.getInt() & 0x00000000ffffffffL; + currentData.position(currentData.position() + (int) keyLength); + long valueLength = currentData.getInt() & 0x00000000ffffffffL; + currentData.position(currentData.position() + (int) valueLength); + } + // Infer the length of the payload from position... + payloadLength = payloadLength + currentData.position(); + + // Reset to mark (i.e. skip frame headers) prior to getting the data + currentData.reset(); + + // get the data, shift mark and compact so next iteration can + // read rest of buffer. + byte[] bytes = new byte[payloadLength]; + currentData.get(bytes, 0, payloadLength); + currentData.mark(); + + // Add payload to frame + internalFrameBuilder.payload(bytes); + + // data frame is created + LumberjackFrame frame = internalFrameBuilder.build(); + frames.add(frame); + internalFrameBuilder.reset(); + } + + return frames; + } + + + private void processVERSION(final byte b) { + byte version = b; + frameBuilder.version(version); + logger.debug("Version number is {}", new Object[]{version}); + currBytes.write(b); + currState = LumberjackState.FRAMETYPE; + } + + private void processFRAMETYPE(final byte b) { + decodedFrameType = b; + frameBuilder.frameType(decodedFrameType); + logger.debug("Frame type is {}", new Object[]{decodedFrameType}); + currBytes.write(b); + currState = LumberjackState.PAYLOAD; + } + + private void processPAYLOAD(final byte b) { + currBytes.write(b); + switch (decodedFrameType) { + case FRAME_WINDOWSIZE: //'W' + if (currBytes.size() < 6) { + logger.trace("Lumberjack currBytes contents are {}", currBytes.toString()); + break; + } else if (currBytes.size() == 6) { + frameBuilder.dataSize = ByteBuffer.wrap(java.util.Arrays.copyOfRange(currBytes.toByteArray(), 2, 6)).getInt() & 0x00000000ffffffffL; + logger.debug("Data size is {}", new Object[]{frameBuilder.dataSize}); + // Sets payload to empty as frame contains no data + frameBuilder.payload(new byte[]{}); + currBytes.reset(); + currState = LumberjackState.COMPLETE; + windowSize = frameBuilder.dataSize; + break; + } else { + break; + } + case FRAME_COMPRESSED: //'C' + if (currBytes.size() < 6) { + logger.trace("Lumberjack currBytes contents are {}", currBytes.toString()); + break; + } else if (currBytes.size() >= 6) { + frameBuilder.dataSize = ByteBuffer.wrap(java.util.Arrays.copyOfRange(currBytes.toByteArray(), 2, 6)).getInt() & 0x00000000ffffffffL; + if (currBytes.size() - 6 == frameBuilder.dataSize) { + try { + byte[] buf = java.util.Arrays.copyOfRange(currBytes.toByteArray(), 6, currBytes.size()); + InputStream in = new InflaterInputStream(new ByteArrayInputStream(buf)); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + byte[] buffer = new byte[1024]; + int len; + while ((len = in.read(buffer)) > 0) { + out.write(buffer, 0, len); + } + in.close(); + out.close(); + decompressedData = out.toByteArray(); + // buf is no longer needed + buf = null; + logger.debug("Finished decompressing data"); + // Decompression is complete, we should be able to proceed with resetting currBytes and curSrtate and iterating them + // as type 'D' frames + frameBuilder.dataSize(decompressedData.length); + currState = LumberjackState.COMPLETE; + + } catch (IOException e) { + throw new LumberjackFrameException("Error decompressing frame: " + e.getMessage(), e); + } + + } + break; + + // If currentByte.size is not lower than six and also not equal or great than 6... + } else { + break; + } + } + } + + private void processCOMPLETE() { + currBytes.reset(); + frameBuilder.reset(); + currState = LumberjackState.VERSION; + } + +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackEncoder.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackEncoder.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackEncoder.java new file mode 100644 index 0000000..e15ec55 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackEncoder.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.frame; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * Encodes a LumberjackFrame into raw bytes using the given charset. + */ +public class LumberjackEncoder { + + + public byte[] encode(final LumberjackFrame frame) { + final ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + + // Writes the version + buffer.write(frame.getVersion()); + + // Writes the frameType + buffer.write(frame.getFrameType()); + + // Writes the sequence number + try { + buffer.write(frame.getPayload()); + } catch (IOException e) { + throw new LumberjackFrameException("Error decoding Lumberjack frame: " + e.getMessage(), e); + } + + return buffer.toByteArray(); + } + +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrame.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrame.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrame.java new file mode 100644 index 0000000..7fedf38 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrame.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.frame; + + +/** + * A Lumberjack frame received from a channel. + */ +public class LumberjackFrame { + + public static final byte DELIMITER = 10; + + private final byte version; + private final byte frameType; + private final byte[] payload; + private final long dataSize; + private final long seqNumber; + + private LumberjackFrame(final Builder builder) { + this.version = builder.version; + this.frameType = builder.frameType; + this.payload = builder.payload; + this.dataSize = builder.dataSize; + this.seqNumber = builder.seqNumber; + + if (version < 2 || payload.length < 0 ) { + throw new LumberjackFrameException("Invalid Frame"); + } + } + + public long getSeqNumber() { + return seqNumber; + } + + public byte getVersion() { + return version; + } + + public byte getFrameType() { + return frameType; + } + + public byte [] getPayload() { + return payload; + } + + /** + * Builder for a LumberjackFrame. + */ + public static class Builder { + + byte version; + byte frameType; + byte [] payload; + long dataSize; + long seqNumber; + + public Builder() { + reset(); + } + + public void reset() { + version = -1; + seqNumber = -1; + frameType = -1; + payload = null; + } + + public Builder version(final byte version) { + this.version = version; + return this; + } + + public Builder seqNumber(final long seqNumber) { + this.seqNumber = seqNumber; + return this; + } + + public Builder frameType(final byte frameType) { + this.frameType = frameType; + return this; + } + + public Builder dataSize(final long dataSize) { + this.dataSize = dataSize; + return this; + } + + public Builder payload(final byte [] payload) { + this.payload = payload; + return this; + } + + + public LumberjackFrame build() { + return new LumberjackFrame(this); + } + + } + +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrameException.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrameException.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrameException.java new file mode 100644 index 0000000..61392ff --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackFrameException.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.frame; + +/** + * Represents an error encountered when decoding Lumberjack frames. + */ +public class LumberjackFrameException extends RuntimeException { + + public LumberjackFrameException(String message) { + super(message); + } + + public LumberjackFrameException(String message, Throwable cause) { + super(message, cause); + } + +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackState.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackState.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackState.java new file mode 100644 index 0000000..d3f6dd4 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/frame/LumberjackState.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.frame; + +/** + * The parts of a Lumberjack frame. + */ +public enum LumberjackState { + + VERSION, + FRAMETYPE, + PAYLOAD, + COMPLETE +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackFrameHandler.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackFrameHandler.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackFrameHandler.java new file mode 100644 index 0000000..40c41ce --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackFrameHandler.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.handler; + + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.SelectionKey; +import java.nio.channels.SocketChannel; +import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.BlockingQueue; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher; +import org.apache.nifi.processor.util.listen.event.Event; +import org.apache.nifi.processor.util.listen.event.EventFactory; +import org.apache.nifi.processor.util.listen.event.EventFactoryUtil; +import org.apache.nifi.processor.util.listen.event.EventQueue; +import org.apache.nifi.processor.util.listen.response.ChannelResponder; +import org.apache.nifi.processors.lumberjack.event.LumberjackMetadata; +import org.apache.nifi.processors.lumberjack.frame.LumberjackEncoder; +import org.apache.nifi.processors.lumberjack.frame.LumberjackFrame; + +import com.google.gson.Gson; + +/** + * Encapsulates the logic to handle a LumberjackFrame once it has been read from the channel. + */ +public class LumberjackFrameHandler<E extends Event<SocketChannel>> { + + private final Charset charset; + private final EventFactory<E> eventFactory; + private final EventQueue<E> events; + private final SelectionKey key; + private final AsyncChannelDispatcher dispatcher; + private final LumberjackEncoder encoder; + private final ComponentLog logger; + + public LumberjackFrameHandler(final SelectionKey selectionKey, + final Charset charset, + final EventFactory<E> eventFactory, + final BlockingQueue<E> events, + final AsyncChannelDispatcher dispatcher, + final ComponentLog logger) { + this.key = selectionKey; + this.charset = charset; + this.eventFactory = eventFactory; + this.dispatcher = dispatcher; + this.logger = logger; + this.events = new EventQueue<>(events, logger); + this.encoder = new LumberjackEncoder(); + } + + public void handle(final LumberjackFrame frame, final ChannelResponder<SocketChannel> responder, final String sender) + throws IOException, InterruptedException { + + final Map<String, String> metadata = EventFactoryUtil.createMapWithSender(sender.toString()); + metadata.put(LumberjackMetadata.SEQNUMBER_KEY, String.valueOf(frame.getSeqNumber())); + String line = ""; + + /* If frameType is a data Frame, Handle the Lumberjack data payload, iterating over it and extracting + keys and values into metadata. + + All keys are inserted into metadata with the exception of line that gets added into the body of the event + */ + if (frame.getFrameType() == 0x44) { + ByteBuffer currentData = ByteBuffer.wrap(frame.getPayload()); + long pairCount = currentData.getInt() & 0x00000000ffffffffL; + Map<String,String> fields = new HashMap<>(); + for (int i = 0; i < pairCount; i++) { + long keyLength = currentData.getInt() & 0x00000000ffffffffL; + byte[] bytes = new byte[(int) keyLength]; + currentData.get(bytes); + String key = new String(bytes); + long valueLength = currentData.getInt() & 0x00000000ffffffffL; + bytes = new byte[(int) valueLength]; + currentData.get(bytes); + String value = new String(bytes); + + if (key.equals("line")) { + line = value; + } else { + fields.put(key, value); + } + } + // Serialize the fields into a String to push it via metdate + Gson serialFields = new Gson(); + + metadata.put("lumberjack.fields", serialFields.toJson(fields).toString()); + + // queue the raw event blocking until space is available, reset the buffer + final E event = eventFactory.create(line.getBytes(), metadata, responder); + events.offer(event); + } else if (frame.getFrameType() == 0x4A ) { + logger.error("Data type was JSON. JSON payload aren't yet supported, pending the documentation of Lumberjack protocol v2"); + } + } + } http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSSLSocketChannelHandler.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSSLSocketChannelHandler.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSSLSocketChannelHandler.java new file mode 100644 index 0000000..7a6fdf1 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSSLSocketChannelHandler.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.handler; + +import java.io.IOException; +import java.net.InetAddress; +import java.nio.channels.SelectionKey; +import java.nio.channels.SocketChannel; +import java.nio.charset.Charset; +import java.util.List; +import java.util.concurrent.BlockingQueue; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher; +import org.apache.nifi.processor.util.listen.event.Event; +import org.apache.nifi.processor.util.listen.event.EventFactory; +import org.apache.nifi.processor.util.listen.handler.socket.SSLSocketChannelHandler; +import org.apache.nifi.processor.util.listen.response.socket.SSLSocketChannelResponder; +import org.apache.nifi.processors.lumberjack.frame.LumberjackDecoder; +import org.apache.nifi.processors.lumberjack.frame.LumberjackFrame; +import org.apache.nifi.processors.lumberjack.frame.LumberjackFrameException; +import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel; + +/** + * A Lumberjack implementation of SSLSocketChannelHandler. + */ +public class LumberjackSSLSocketChannelHandler<E extends Event<SocketChannel>> extends SSLSocketChannelHandler<E> { + + private LumberjackDecoder decoder; + private LumberjackFrameHandler<E> frameHandler; + + public LumberjackSSLSocketChannelHandler(final SelectionKey key, + final AsyncChannelDispatcher dispatcher, + final Charset charset, + final EventFactory<E> eventFactory, + final BlockingQueue<E> events, + final ComponentLog logger) { + super(key, dispatcher, charset, eventFactory, events, logger); + this.decoder = new LumberjackDecoder(charset); + this.frameHandler = new LumberjackFrameHandler<>(key, charset, eventFactory, events, dispatcher, logger); + } + + @Override + protected void processBuffer(final SSLSocketChannel sslSocketChannel, final SocketChannel socketChannel, + final int bytesRead, final byte[] buffer) throws InterruptedException, IOException { + + final InetAddress sender = socketChannel.socket().getInetAddress(); + try { + + // go through the buffer parsing the Lumberjack command + for (int i = 0; i < bytesRead; i++) { + byte currByte = buffer[i]; + + // if we found the end of a frame, handle the frame and mark the buffer + if (decoder.process(currByte)) { + final List<LumberjackFrame> frames = decoder.getFrames(); + // A list of events has been generated + for (LumberjackFrame frame : frames) { + logger.debug("Received Lumberjack frame with transaction {} and command {}", + new Object[]{frame.getSeqNumber(), frame.getSeqNumber()}); + // Ignore the WINDOWS type frames as they contain no payload. + if (frame.getFrameType() != 0x57 ) { + final SSLSocketChannelResponder responder = new SSLSocketChannelResponder(socketChannel, sslSocketChannel); + frameHandler.handle(frame, responder, sender.toString()); + } + } + } + } + + logger.debug("Done processing buffer"); + + } catch (final LumberjackFrameException rfe) { + logger.error("Error reading Lumberjack frames due to {}", new Object[] {rfe.getMessage()} , rfe); + // if an invalid frame or bad data was sent then the decoder will be left in a + // corrupted state, so lets close the connection and cause the client to re-establish + dispatcher.completeConnection(key); + } + } + +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandler.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandler.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandler.java new file mode 100644 index 0000000..2621222 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandler.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.handler; + +import java.io.IOException; +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.nio.channels.SelectionKey; +import java.nio.channels.SocketChannel; +import java.nio.charset.Charset; +import java.util.List; +import java.util.concurrent.BlockingQueue; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher; +import org.apache.nifi.processor.util.listen.event.Event; +import org.apache.nifi.processor.util.listen.event.EventFactory; +import org.apache.nifi.processor.util.listen.handler.socket.StandardSocketChannelHandler; +import org.apache.nifi.processor.util.listen.response.socket.SocketChannelResponder; +import org.apache.nifi.processors.lumberjack.frame.LumberjackDecoder; +import org.apache.nifi.processors.lumberjack.frame.LumberjackFrame; +import org.apache.nifi.processors.lumberjack.frame.LumberjackFrameException; + +/** + * Extends the StandardSocketChannelHandler to decode bytes into Lumberjack frames. + */ +public class LumberjackSocketChannelHandler<E extends Event<SocketChannel>> extends StandardSocketChannelHandler<E> { + + private LumberjackDecoder decoder; + private LumberjackFrameHandler<E> frameHandler; + + public LumberjackSocketChannelHandler(final SelectionKey key, + final AsyncChannelDispatcher dispatcher, + final Charset charset, + final EventFactory<E> eventFactory, + final BlockingQueue<E> events, + final ComponentLog logger) { + super(key, dispatcher, charset, eventFactory, events, logger); + this.decoder = new LumberjackDecoder(charset); + this.frameHandler = new LumberjackFrameHandler<>(key, charset, eventFactory, events, dispatcher, logger); + } + + @Override + protected void processBuffer(final SocketChannel socketChannel, final ByteBuffer socketBuffer) + throws InterruptedException, IOException { + + // get total bytes in buffer + final int total = socketBuffer.remaining(); + final InetAddress sender = socketChannel.socket().getInetAddress(); + + try { + // go through the buffer parsing the Lumberjack command + for (int i = 0; i < total; i++) { + byte currByte = socketBuffer.get(); + + // if we found the end of a frame, handle the frame and mark the buffer + if (decoder.process(currByte)) { + final List<LumberjackFrame> frames = decoder.getFrames(); + + for (LumberjackFrame frame : frames) { + //TODO: Clean this + logger.debug("Received Lumberjack frame with transaction {} and command {}", + new Object[]{frame.getSeqNumber(), frame.getSeqNumber()}); + // Ignore the WINDOWS type frames as they contain no payload. + if (frame.getFrameType() != 0x57) { + final SocketChannelResponder responder = new SocketChannelResponder(socketChannel); + frameHandler.handle(frame, responder, sender.toString()); + } + } + socketBuffer.mark(); + } + } + logger.debug("Done processing buffer"); + + } catch (final LumberjackFrameException rfe) { + logger.error("Error reading Lumberjack frames due to {}", new Object[] {rfe.getMessage()}, rfe); + // if an invalid frame or bad data was sent then the decoder will be left in a + // corrupted state, so lets close the connection and cause the client to re-establish + dispatcher.completeConnection(key); + } + } + + // not used for anything in Lumberjack since the decoder encapsulates the delimiter + @Override + public byte getDelimiter() { + return LumberjackFrame.DELIMITER; + } + +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandlerFactory.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandlerFactory.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandlerFactory.java new file mode 100644 index 0000000..be34b92 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/handler/LumberjackSocketChannelHandlerFactory.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.handler; + +import java.nio.channels.SelectionKey; +import java.nio.channels.SocketChannel; +import java.nio.charset.Charset; +import java.util.concurrent.BlockingQueue; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher; +import org.apache.nifi.processor.util.listen.event.Event; +import org.apache.nifi.processor.util.listen.event.EventFactory; +import org.apache.nifi.processor.util.listen.handler.ChannelHandler; +import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory; + +/** + * Default factory for creating Lumberjack socket channel handlers. + */ +public class LumberjackSocketChannelHandlerFactory<E extends Event<SocketChannel>> implements ChannelHandlerFactory<E, AsyncChannelDispatcher> { + + @Override + public ChannelHandler<E, AsyncChannelDispatcher> createHandler(final SelectionKey key, + final AsyncChannelDispatcher dispatcher, + final Charset charset, + final EventFactory<E> eventFactory, + final BlockingQueue<E> events, + final ComponentLog logger) { + return new LumberjackSocketChannelHandler<>(key, dispatcher, charset, eventFactory, events, logger); + } + + @Override + public ChannelHandler<E, AsyncChannelDispatcher> createSSLHandler(final SelectionKey key, + final AsyncChannelDispatcher dispatcher, + final Charset charset, + final EventFactory<E> eventFactory, + final BlockingQueue<E> events, + final ComponentLog logger) { + return new LumberjackSSLSocketChannelHandler<>(key, dispatcher, charset, eventFactory, events, logger); + } + +} http://git-wip-us.apache.org/repos/asf/nifi/blob/5a897915/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/response/LumberjackChannelResponse.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/response/LumberjackChannelResponse.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/response/LumberjackChannelResponse.java new file mode 100644 index 0000000..8749759 --- /dev/null +++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/response/LumberjackChannelResponse.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.lumberjack.response; + +import org.apache.nifi.processor.util.listen.response.ChannelResponse; +import org.apache.nifi.processors.lumberjack.frame.LumberjackFrame; +import org.apache.nifi.processors.lumberjack.frame.LumberjackEncoder; + +/** + * Creates a LumberjackFrame for the provided response and returns the encoded frame. + */ +public class LumberjackChannelResponse implements ChannelResponse { + + private final LumberjackEncoder encoder; + private final LumberjackResponse response; + + public LumberjackChannelResponse(final LumberjackEncoder encoder, final LumberjackResponse response) { + this.encoder = encoder; + this.response = response; + } + + @Override + public byte[] toByteArray() { + final LumberjackFrame frame = response.toFrame(); + return encoder.encode(frame); + } + +}