mirror of https://github.com/apache/nifi.git
NIFI-13509 Removed ListenTCPRecord, ListenRELP and others
- Removed ConvertJSONToSql - Removed ParseCEF - Removed nifi-security-socket-ssl - Removed Socket Dispatcher Reader classes This closes #9045 Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
parent
4ff70dd233
commit
f1fb106bf8
|
@ -161,7 +161,6 @@ MiNiFi is able to use the following processors out of the box:
|
|||
* CompressContent
|
||||
* ControlRate
|
||||
* ConvertCharacterSet
|
||||
* ConvertJSONToSQL
|
||||
* ConvertRecord
|
||||
* CountText
|
||||
* CryptographicHashContent
|
||||
|
@ -208,10 +207,8 @@ MiNiFi is able to use the following processors out of the box:
|
|||
* ListSFTP
|
||||
* ListenFTP
|
||||
* ListenHTTP
|
||||
* ListenRELP
|
||||
* ListenSyslog
|
||||
* ListenTCP
|
||||
* ListenTCPRecord
|
||||
* ListenUDP
|
||||
* ListenUDPRecord
|
||||
* LogAttribute
|
||||
|
@ -224,7 +221,6 @@ MiNiFi is able to use the following processors out of the box:
|
|||
* MonitorActivity
|
||||
* Notify
|
||||
* PackageFlowFile
|
||||
* ParseCEF
|
||||
* ParseSyslog
|
||||
* ParseSyslog5424
|
||||
* PartitionRecord
|
||||
|
@ -281,7 +277,6 @@ If you want to create a dataflow with a processor not shipped with MiNiFi, you c
|
|||
|
||||
**Note:** The following processors are also a part of the default distribution but require adding a NAR for a Controller Service not packaged by default. The processors are grouped by the NAR that is required.
|
||||
* nifi-dbcp-service-nar
|
||||
* ConvertJSONToSQL
|
||||
* PutSQL
|
||||
* GenerateTableFetch
|
||||
* ListDatabaseTables
|
||||
|
|
|
@ -352,11 +352,6 @@
|
|||
<artifactId>nifi-security-kerberos-api</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-socket-ssl</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-ssl</artifactId>
|
||||
|
|
|
@ -1,72 +0,0 @@
|
|||
<?xml version="1.0"?>
|
||||
<!--
|
||||
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 https://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-commons</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>nifi-security-socket-ssl</artifactId>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-utils</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-utils</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-utils-api</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-handler</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-buffer</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-common</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-codec</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-transport</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -1,731 +0,0 @@
|
|||
/*
|
||||
* 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.remote.io.socket.ssl;
|
||||
|
||||
import org.apache.nifi.remote.exception.TransmissionDisabledException;
|
||||
import org.apache.nifi.remote.io.socket.BufferStateManager;
|
||||
import org.apache.nifi.remote.io.socket.BufferStateManager.Direction;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import javax.net.ssl.SSLEngineResult;
|
||||
import javax.net.ssl.SSLEngineResult.Status;
|
||||
import javax.net.ssl.SSLException;
|
||||
import javax.net.ssl.SSLHandshakeException;
|
||||
import javax.net.ssl.SSLSession;
|
||||
import java.io.Closeable;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketAddress;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* SSLSocketChannel supports reading and writing bytes using TLS and NIO SocketChannels with configurable timeouts
|
||||
*/
|
||||
public class SSLSocketChannel implements Closeable {
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(SSLSocketChannel.class);
|
||||
|
||||
private static final int MINIMUM_READ_BUFFER_SIZE = 1;
|
||||
private static final int DISCARD_BUFFER_LENGTH = 8192;
|
||||
private static final int END_OF_STREAM = -1;
|
||||
private static final byte[] EMPTY_MESSAGE = new byte[0];
|
||||
private static final long BUFFER_FULL_EMPTY_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(1, TimeUnit.MILLISECONDS);
|
||||
private static final long FINISH_CONNECT_SLEEP = 50;
|
||||
private static final long INITIAL_INCREMENTAL_SLEEP = 1;
|
||||
private static final boolean CLIENT_AUTHENTICATION_REQUIRED = true;
|
||||
|
||||
private final String remoteAddress;
|
||||
private final int port;
|
||||
private final SSLEngine engine;
|
||||
private final SocketAddress socketAddress;
|
||||
private final BufferStateManager streamInManager;
|
||||
private final BufferStateManager streamOutManager;
|
||||
private final BufferStateManager appDataManager;
|
||||
private final SocketChannel channel;
|
||||
private int timeoutMillis = 30000;
|
||||
|
||||
private volatile boolean interrupted = false;
|
||||
private volatile ChannelStatus channelStatus = ChannelStatus.DISCONNECTED;
|
||||
|
||||
/**
|
||||
* SSLSocketChannel constructor with SSLContext and remote address parameters
|
||||
*
|
||||
* @param sslContext SSLContext used to create SSLEngine with specified client mode
|
||||
* @param remoteAddress Remote Address used for connection
|
||||
* @param port Remote Port used for connection
|
||||
* @param bindAddress Local address used for binding server channel when provided
|
||||
* @param useClientMode Use Client Mode
|
||||
* @throws IOException Thrown on failures creating Socket Channel
|
||||
*/
|
||||
public SSLSocketChannel(final SSLContext sslContext, final String remoteAddress, final int port, final InetAddress bindAddress, final boolean useClientMode) throws IOException {
|
||||
this.engine = createEngine(sslContext, useClientMode);
|
||||
this.channel = createSocketChannel(bindAddress);
|
||||
this.socketAddress = new InetSocketAddress(remoteAddress, port);
|
||||
this.remoteAddress = remoteAddress;
|
||||
this.port = port;
|
||||
|
||||
streamInManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
|
||||
streamOutManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
|
||||
appDataManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getApplicationBufferSize()));
|
||||
}
|
||||
|
||||
/**
|
||||
* SSLSocketChannel constructor with SSLContext and connected SocketChannel
|
||||
*
|
||||
* @param sslContext SSLContext used to create SSLEngine with specified client mode
|
||||
* @param socketChannel Connected SocketChannel
|
||||
* @param useClientMode Use Client Mode
|
||||
* @throws IOException Thrown on SocketChannel.getRemoteAddress()
|
||||
*/
|
||||
public SSLSocketChannel(final SSLContext sslContext, final SocketChannel socketChannel, final boolean useClientMode) throws IOException {
|
||||
this(createEngine(sslContext, useClientMode), socketChannel);
|
||||
}
|
||||
|
||||
/**
|
||||
* SSLSocketChannel constructor with configured SSLEngine and connected SocketChannel
|
||||
*
|
||||
* @param sslEngine SSLEngine configured with mode and client authentication
|
||||
* @param socketChannel Connected SocketChannel
|
||||
* @throws IOException Thrown on SocketChannel.getRemoteAddress()
|
||||
*/
|
||||
public SSLSocketChannel(final SSLEngine sslEngine, final SocketChannel socketChannel) throws IOException {
|
||||
if (!socketChannel.isConnected()) {
|
||||
throw new IllegalArgumentException("Connected SocketChannel required");
|
||||
}
|
||||
|
||||
socketChannel.configureBlocking(false);
|
||||
this.channel = socketChannel;
|
||||
this.socketAddress = socketChannel.getRemoteAddress();
|
||||
final Socket socket = socketChannel.socket();
|
||||
this.remoteAddress = socket.getInetAddress().toString();
|
||||
this.port = socket.getPort();
|
||||
this.engine = sslEngine;
|
||||
|
||||
streamInManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
|
||||
streamOutManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize()));
|
||||
appDataManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getApplicationBufferSize()));
|
||||
}
|
||||
|
||||
public void setTimeout(final int timeoutMillis) {
|
||||
this.timeoutMillis = timeoutMillis;
|
||||
}
|
||||
|
||||
public int getTimeout() {
|
||||
return timeoutMillis;
|
||||
}
|
||||
|
||||
/**
|
||||
* Connect Channel when not connected and perform handshake process
|
||||
*
|
||||
* @throws IOException Thrown on connection failures
|
||||
*/
|
||||
public void connect() throws IOException {
|
||||
channelStatus = ChannelStatus.CONNECTING;
|
||||
|
||||
try {
|
||||
if (!channel.isConnected()) {
|
||||
logOperation("Connection Started");
|
||||
final long started = System.currentTimeMillis();
|
||||
|
||||
if (!channel.connect(socketAddress)) {
|
||||
while (!channel.finishConnect()) {
|
||||
checkInterrupted();
|
||||
checkTimeoutExceeded(started);
|
||||
|
||||
try {
|
||||
TimeUnit.MILLISECONDS.sleep(FINISH_CONNECT_SLEEP);
|
||||
} catch (final InterruptedException e) {
|
||||
logOperation("Connection Interrupted");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
channelStatus = ChannelStatus.CONNECTED;
|
||||
} catch (final Exception e) {
|
||||
close();
|
||||
throw new SSLException(String.format("[%s:%d] Connection Failed", remoteAddress, port), e);
|
||||
}
|
||||
|
||||
try {
|
||||
performHandshake();
|
||||
} catch (final IOException e) {
|
||||
close();
|
||||
throw new SSLException(String.format("[%s:%d] Handshake Failed", remoteAddress, port), e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown Socket Channel input and read available bytes
|
||||
*
|
||||
* @throws IOException Thrown on Socket Channel failures
|
||||
*/
|
||||
public void consume() throws IOException {
|
||||
channel.shutdownInput();
|
||||
|
||||
final byte[] byteBuffer = new byte[DISCARD_BUFFER_LENGTH];
|
||||
final ByteBuffer buffer = ByteBuffer.wrap(byteBuffer);
|
||||
int readCount;
|
||||
do {
|
||||
readCount = channel.read(buffer);
|
||||
buffer.flip();
|
||||
} while (readCount > 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Is Channel Closed
|
||||
*
|
||||
* @return Channel Closed Status
|
||||
*/
|
||||
public boolean isClosed() {
|
||||
if (ChannelStatus.CLOSED == channelStatus) {
|
||||
return true;
|
||||
}
|
||||
|
||||
// Read Channel to determine closed status
|
||||
final ByteBuffer inputBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize());
|
||||
int bytesRead;
|
||||
try {
|
||||
bytesRead = channel.read(inputBuffer);
|
||||
} catch (final IOException e) {
|
||||
LOGGER.warn("[{}:{}] Closed Status Read Failed", remoteAddress, port, e);
|
||||
bytesRead = END_OF_STREAM;
|
||||
}
|
||||
logOperationBytes("Closed Status Read", bytesRead);
|
||||
|
||||
if (bytesRead == 0) {
|
||||
return false;
|
||||
} else if (bytesRead > 0) {
|
||||
try {
|
||||
final SSLEngineResult unwrapResult = unwrap();
|
||||
if (Status.CLOSED == unwrapResult.getStatus()) {
|
||||
readChannelDiscard();
|
||||
engine.closeInbound();
|
||||
} else {
|
||||
streamInManager.compact();
|
||||
return false;
|
||||
}
|
||||
} catch (final IOException e) {
|
||||
LOGGER.warn("[{}:{}] Closed Status Unwrap Failed", remoteAddress, port, e);
|
||||
}
|
||||
}
|
||||
|
||||
// Close Channel when encountering end of stream or closed status
|
||||
try {
|
||||
close();
|
||||
} catch (final IOException e) {
|
||||
LOGGER.warn("[{}:{}] Close Failed", remoteAddress, port, e);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Close Channel and process notifications
|
||||
*
|
||||
* @throws IOException Thrown on SSLEngine.wrap() failures
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
logOperation("Close Requested");
|
||||
if (channelStatus == ChannelStatus.CLOSED) {
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
engine.closeOutbound();
|
||||
|
||||
streamOutManager.clear();
|
||||
final ByteBuffer inputBuffer = ByteBuffer.wrap(EMPTY_MESSAGE);
|
||||
final ByteBuffer outputBuffer = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
|
||||
SSLEngineResult wrapResult = wrap(inputBuffer, outputBuffer);
|
||||
Status status = wrapResult.getStatus();
|
||||
if (Status.OK == status) {
|
||||
logOperation("Clearing Outbound Buffer");
|
||||
outputBuffer.clear();
|
||||
wrapResult = wrap(inputBuffer, outputBuffer);
|
||||
status = wrapResult.getStatus();
|
||||
}
|
||||
if (Status.CLOSED == status) {
|
||||
final ByteBuffer streamOutputBuffer = streamOutManager.prepareForRead(MINIMUM_READ_BUFFER_SIZE);
|
||||
try {
|
||||
writeChannel(streamOutputBuffer);
|
||||
} catch (final IOException e) {
|
||||
logOperation(String.format("Write Close Notification Failed: %s", e.getMessage()));
|
||||
}
|
||||
} else {
|
||||
throw new SSLException(String.format("[%s:%d] Invalid Wrap Result Status [%s]", remoteAddress, port, status));
|
||||
}
|
||||
} finally {
|
||||
channelStatus = ChannelStatus.CLOSED;
|
||||
readChannelDiscard();
|
||||
closeQuietly(channel.socket());
|
||||
closeQuietly(channel);
|
||||
logOperation("Close Completed");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get application bytes available for reading
|
||||
*
|
||||
* @return Number of application bytes available for reading
|
||||
* @throws IOException Thrown on failures checking for available bytes
|
||||
*/
|
||||
public int available() throws IOException {
|
||||
final ByteBuffer appDataBuffer = appDataManager.prepareForRead(MINIMUM_READ_BUFFER_SIZE);
|
||||
return appDataBuffer.remaining();
|
||||
}
|
||||
|
||||
/**
|
||||
* Read and return one byte
|
||||
*
|
||||
* @return Byte read or -1 when end of stream reached
|
||||
* @throws IOException Thrown on read failures
|
||||
*/
|
||||
public int read() throws IOException {
|
||||
final byte[] buffer = new byte[1];
|
||||
|
||||
final int bytesRead = read(buffer);
|
||||
if (bytesRead == END_OF_STREAM) {
|
||||
return END_OF_STREAM;
|
||||
}
|
||||
|
||||
return Byte.toUnsignedInt(buffer[0]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read available bytes into buffer
|
||||
*
|
||||
* @param buffer Byte array buffer
|
||||
* @return Number of bytes read
|
||||
* @throws IOException Thrown on read failures
|
||||
*/
|
||||
public int read(final byte[] buffer) throws IOException {
|
||||
return read(buffer, 0, buffer.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read available bytes into buffer based on offset and length requested
|
||||
*
|
||||
* @param buffer Byte array buffer
|
||||
* @param offset Buffer offset
|
||||
* @param len Length of bytes to read
|
||||
* @return Number of bytes read
|
||||
* @throws IOException Thrown on read failures
|
||||
*/
|
||||
public int read(final byte[] buffer, final int offset, final int len) throws IOException {
|
||||
logOperationBytes("Read Requested", len);
|
||||
checkChannelStatus();
|
||||
|
||||
int applicationBytesRead = readApplicationBuffer(buffer, offset, len);
|
||||
if (applicationBytesRead > 0) {
|
||||
return applicationBytesRead;
|
||||
}
|
||||
appDataManager.clear();
|
||||
|
||||
final SSLEngineResult unwrapResult = unwrapBufferReadChannel();
|
||||
final Status status = unwrapResult.getStatus();
|
||||
if (Status.CLOSED == status) {
|
||||
applicationBytesRead = readApplicationBuffer(buffer, offset, len);
|
||||
if (applicationBytesRead == 0) {
|
||||
return END_OF_STREAM;
|
||||
}
|
||||
streamInManager.compact();
|
||||
return applicationBytesRead;
|
||||
} else if (Status.OK == status) {
|
||||
applicationBytesRead = readApplicationBuffer(buffer, offset, len);
|
||||
if (applicationBytesRead == 0) {
|
||||
throw new IOException("Read Application Buffer Failed");
|
||||
}
|
||||
streamInManager.compact();
|
||||
return applicationBytesRead;
|
||||
} else {
|
||||
throw new IllegalStateException(String.format("SSLEngineResult Status [%s] not expected from unwrap", status));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write one byte to channel
|
||||
*
|
||||
* @param data Byte to be written
|
||||
* @throws IOException Thrown on write failures
|
||||
*/
|
||||
public void write(final int data) throws IOException {
|
||||
write(new byte[]{(byte) data}, 0, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write bytes to channel
|
||||
*
|
||||
* @param data Byte array to be written
|
||||
* @throws IOException Thrown on write failures
|
||||
*/
|
||||
public void write(final byte[] data) throws IOException {
|
||||
write(data, 0, data.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write data to channel performs multiple iterations based on data length
|
||||
*
|
||||
* @param data Byte array to be written
|
||||
* @param offset Byte array offset
|
||||
* @param len Length of bytes for writing
|
||||
* @throws IOException Thrown on write failures
|
||||
*/
|
||||
public void write(final byte[] data, final int offset, final int len) throws IOException {
|
||||
logOperationBytes("Write Started", len);
|
||||
checkChannelStatus();
|
||||
|
||||
final int applicationBufferSize = engine.getSession().getApplicationBufferSize();
|
||||
logOperationBytes("Write Application Buffer Size", applicationBufferSize);
|
||||
int iterations = len / applicationBufferSize;
|
||||
if (len % applicationBufferSize > 0) {
|
||||
iterations++;
|
||||
}
|
||||
|
||||
for (int i = 0; i < iterations; i++) {
|
||||
streamOutManager.clear();
|
||||
final int itrOffset = offset + i * applicationBufferSize;
|
||||
final int itrLen = Math.min(len - itrOffset, applicationBufferSize);
|
||||
final ByteBuffer byteBuffer = ByteBuffer.wrap(data, itrOffset, itrLen);
|
||||
|
||||
final BufferStateManager bufferStateManager = new BufferStateManager(byteBuffer, Direction.READ);
|
||||
final Status status = wrapWriteChannel(bufferStateManager);
|
||||
switch (status) {
|
||||
case BUFFER_OVERFLOW:
|
||||
streamOutManager.ensureSize(engine.getSession().getPacketBufferSize());
|
||||
appDataManager.ensureSize(engine.getSession().getApplicationBufferSize());
|
||||
continue;
|
||||
case OK:
|
||||
continue;
|
||||
case CLOSED:
|
||||
throw new IOException("Channel is closed");
|
||||
case BUFFER_UNDERFLOW:
|
||||
throw new AssertionError("Got Buffer Underflow but should not have...");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Interrupt processing and disable transmission
|
||||
*/
|
||||
public void interrupt() {
|
||||
this.interrupted = true;
|
||||
}
|
||||
|
||||
private void performHandshake() throws IOException {
|
||||
logOperation("Handshake Started");
|
||||
channelStatus = ChannelStatus.HANDSHAKING;
|
||||
engine.beginHandshake();
|
||||
|
||||
SSLEngineResult.HandshakeStatus handshakeStatus = engine.getHandshakeStatus();
|
||||
while (true) {
|
||||
logHandshakeStatus(handshakeStatus);
|
||||
|
||||
switch (handshakeStatus) {
|
||||
case FINISHED:
|
||||
case NOT_HANDSHAKING:
|
||||
channelStatus = ChannelStatus.ESTABLISHED;
|
||||
final SSLSession session = engine.getSession();
|
||||
LOGGER.debug("[{}:{}] [{}] Negotiated Protocol [{}] Cipher Suite [{}]",
|
||||
remoteAddress,
|
||||
port,
|
||||
channelStatus,
|
||||
session.getProtocol(),
|
||||
session.getCipherSuite()
|
||||
);
|
||||
return;
|
||||
case NEED_TASK:
|
||||
runDelegatedTasks();
|
||||
handshakeStatus = engine.getHandshakeStatus();
|
||||
break;
|
||||
case NEED_UNWRAP:
|
||||
final SSLEngineResult unwrapResult = unwrapBufferReadChannel();
|
||||
handshakeStatus = unwrapResult.getHandshakeStatus();
|
||||
if (unwrapResult.getStatus() == Status.CLOSED) {
|
||||
throw getHandshakeException(handshakeStatus, "Channel Closed");
|
||||
}
|
||||
streamInManager.compact();
|
||||
appDataManager.clear();
|
||||
break;
|
||||
case NEED_WRAP:
|
||||
final ByteBuffer outboundBuffer = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
|
||||
final SSLEngineResult wrapResult = wrap(ByteBuffer.wrap(EMPTY_MESSAGE), outboundBuffer);
|
||||
handshakeStatus = wrapResult.getHandshakeStatus();
|
||||
final Status wrapResultStatus = wrapResult.getStatus();
|
||||
|
||||
if (wrapResultStatus == Status.BUFFER_OVERFLOW) {
|
||||
streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
|
||||
} else if (wrapResultStatus == Status.OK) {
|
||||
final ByteBuffer streamBuffer = streamOutManager.prepareForRead(MINIMUM_READ_BUFFER_SIZE);
|
||||
final int bytesRemaining = streamBuffer.remaining();
|
||||
writeChannel(streamBuffer);
|
||||
logOperationBytes("Handshake Channel Write Completed", bytesRemaining);
|
||||
streamOutManager.clear();
|
||||
} else {
|
||||
throw getHandshakeException(handshakeStatus, String.format("Wrap Failed [%s]", wrapResult.getStatus()));
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private SSLEngineResult unwrapBufferReadChannel() throws IOException {
|
||||
SSLEngineResult unwrapResult = unwrap();
|
||||
|
||||
while (Status.BUFFER_UNDERFLOW == unwrapResult.getStatus()) {
|
||||
final int channelBytesRead = readChannel();
|
||||
if (channelBytesRead == END_OF_STREAM) {
|
||||
throw new EOFException("End of Stream found for Channel Read");
|
||||
}
|
||||
|
||||
unwrapResult = unwrap();
|
||||
if (SSLEngineResult.HandshakeStatus.FINISHED == unwrapResult.getHandshakeStatus()) {
|
||||
// RFC 8446 Section 4.6 describes Post-Handshake Messages for TLS 1.3
|
||||
logOperation("Processing Post-Handshake Messages");
|
||||
unwrapResult = unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
return unwrapResult;
|
||||
}
|
||||
|
||||
private int readChannel() throws IOException {
|
||||
logOperation("Channel Read Started");
|
||||
final ByteBuffer outputBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize());
|
||||
|
||||
final long started = System.currentTimeMillis();
|
||||
long sleepNanoseconds = INITIAL_INCREMENTAL_SLEEP;
|
||||
while (true) {
|
||||
checkInterrupted();
|
||||
|
||||
if (outputBuffer.remaining() == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
final int channelBytesRead = channel.read(outputBuffer);
|
||||
if (channelBytesRead == 0) {
|
||||
checkTimeoutExceeded(started);
|
||||
sleepNanoseconds = incrementalSleep(sleepNanoseconds);
|
||||
continue;
|
||||
}
|
||||
|
||||
logOperationBytes("Channel Read Completed", channelBytesRead);
|
||||
return channelBytesRead;
|
||||
}
|
||||
}
|
||||
|
||||
private void readChannelDiscard() {
|
||||
try {
|
||||
final ByteBuffer readBuffer = ByteBuffer.allocate(DISCARD_BUFFER_LENGTH);
|
||||
int bytesRead = channel.read(readBuffer);
|
||||
while (bytesRead > 0) {
|
||||
readBuffer.clear();
|
||||
bytesRead = channel.read(readBuffer);
|
||||
}
|
||||
} catch (final IOException e) {
|
||||
LOGGER.debug("[{}:{}] Read Channel Discard Failed", remoteAddress, port, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void writeChannel(final ByteBuffer inputBuffer) throws IOException {
|
||||
long lastWriteCompleted = System.currentTimeMillis();
|
||||
|
||||
int totalBytes = 0;
|
||||
long sleepNanoseconds = INITIAL_INCREMENTAL_SLEEP;
|
||||
while (inputBuffer.hasRemaining()) {
|
||||
checkInterrupted();
|
||||
|
||||
final int written = channel.write(inputBuffer);
|
||||
totalBytes += written;
|
||||
|
||||
if (written > 0) {
|
||||
lastWriteCompleted = System.currentTimeMillis();
|
||||
} else {
|
||||
checkTimeoutExceeded(lastWriteCompleted);
|
||||
sleepNanoseconds = incrementalSleep(sleepNanoseconds);
|
||||
}
|
||||
}
|
||||
|
||||
logOperationBytes("Channel Write Completed", totalBytes);
|
||||
}
|
||||
|
||||
private long incrementalSleep(final long nanoseconds) throws IOException {
|
||||
try {
|
||||
TimeUnit.NANOSECONDS.sleep(nanoseconds);
|
||||
} catch (final InterruptedException e) {
|
||||
close();
|
||||
Thread.currentThread().interrupt();
|
||||
throw new ClosedByInterruptException();
|
||||
}
|
||||
return Math.min(nanoseconds * 2, BUFFER_FULL_EMPTY_WAIT_NANOS);
|
||||
}
|
||||
|
||||
private int readApplicationBuffer(final byte[] buffer, final int offset, final int len) {
|
||||
logOperationBytes("Application Buffer Read Requested", len);
|
||||
final ByteBuffer appDataBuffer = appDataManager.prepareForRead(len);
|
||||
|
||||
final int appDataRemaining = appDataBuffer.remaining();
|
||||
logOperationBytes("Application Buffer Remaining", appDataRemaining);
|
||||
if (appDataRemaining > 0) {
|
||||
final int bytesToCopy = Math.min(len, appDataBuffer.remaining());
|
||||
appDataBuffer.get(buffer, offset, bytesToCopy);
|
||||
|
||||
final int bytesCopied = appDataRemaining - appDataBuffer.remaining();
|
||||
logOperationBytes("Application Buffer Copied", bytesCopied);
|
||||
return bytesCopied;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
private Status wrapWriteChannel(final BufferStateManager inputManager) throws IOException {
|
||||
final ByteBuffer inputBuffer = inputManager.prepareForRead(0);
|
||||
final ByteBuffer outputBuffer = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
|
||||
|
||||
logOperationBytes("Wrap Started", inputBuffer.remaining());
|
||||
Status status = Status.OK;
|
||||
while (inputBuffer.remaining() > 0) {
|
||||
final SSLEngineResult result = wrap(inputBuffer, outputBuffer);
|
||||
status = result.getStatus();
|
||||
if (status == Status.OK) {
|
||||
final ByteBuffer readableOutBuff = streamOutManager.prepareForRead(0);
|
||||
writeChannel(readableOutBuff);
|
||||
streamOutManager.clear();
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
private SSLEngineResult wrap(final ByteBuffer inputBuffer, final ByteBuffer outputBuffer) throws SSLException {
|
||||
final SSLEngineResult result = engine.wrap(inputBuffer, outputBuffer);
|
||||
logEngineResult(result, "WRAP Completed");
|
||||
return result;
|
||||
}
|
||||
|
||||
private SSLEngineResult unwrap() throws IOException {
|
||||
final ByteBuffer streamBuffer = streamInManager.prepareForRead(engine.getSession().getPacketBufferSize());
|
||||
final ByteBuffer applicationBuffer = appDataManager.prepareForWrite(engine.getSession().getApplicationBufferSize());
|
||||
final SSLEngineResult result = engine.unwrap(streamBuffer, applicationBuffer);
|
||||
logEngineResult(result, "UNWRAP Completed");
|
||||
return result;
|
||||
}
|
||||
|
||||
private void runDelegatedTasks() {
|
||||
Runnable delegatedTask;
|
||||
while ((delegatedTask = engine.getDelegatedTask()) != null) {
|
||||
logOperation("Running Delegated Task");
|
||||
delegatedTask.run();
|
||||
}
|
||||
}
|
||||
|
||||
private void closeQuietly(final Closeable closeable) {
|
||||
try {
|
||||
closeable.close();
|
||||
} catch (final Exception e) {
|
||||
logOperation(String.format("Close failed: %s", e.getMessage()));
|
||||
}
|
||||
}
|
||||
|
||||
private SSLHandshakeException getHandshakeException(final SSLEngineResult.HandshakeStatus handshakeStatus, final String message) {
|
||||
final String formatted = String.format("[%s:%d] Handshake Status [%s] %s", remoteAddress, port, handshakeStatus, message);
|
||||
return new SSLHandshakeException(formatted);
|
||||
}
|
||||
|
||||
private void checkChannelStatus() throws IOException {
|
||||
if (ChannelStatus.ESTABLISHED != channelStatus) {
|
||||
connect();
|
||||
}
|
||||
}
|
||||
|
||||
private void checkInterrupted() {
|
||||
if (interrupted) {
|
||||
throw new TransmissionDisabledException();
|
||||
}
|
||||
}
|
||||
|
||||
private void checkTimeoutExceeded(final long started) throws SocketTimeoutException {
|
||||
if (System.currentTimeMillis() > started + timeoutMillis) {
|
||||
throw new SocketTimeoutException(String.format("Timeout Exceeded [%d ms] for [%s:%d]", timeoutMillis, remoteAddress, port));
|
||||
}
|
||||
}
|
||||
|
||||
private void logOperation(final String operation) {
|
||||
LOGGER.trace("[{}:{}] [{}] {}", remoteAddress, port, channelStatus, operation);
|
||||
}
|
||||
|
||||
private void logOperationBytes(final String operation, final int bytes) {
|
||||
LOGGER.trace("[{}:{}] [{}] {} Bytes [{}]", remoteAddress, port, channelStatus, operation, bytes);
|
||||
}
|
||||
|
||||
private void logHandshakeStatus(final SSLEngineResult.HandshakeStatus handshakeStatus) {
|
||||
LOGGER.trace("[{}:{}] [{}] Handshake Status [{}]", remoteAddress, port, channelStatus, handshakeStatus);
|
||||
}
|
||||
|
||||
private void logEngineResult(final SSLEngineResult result, final String method) {
|
||||
LOGGER.trace("[{}:{}] [{}] {} Status [{}] Handshake Status [{}] Produced [{}] Consumed [{}]",
|
||||
remoteAddress,
|
||||
port,
|
||||
channelStatus,
|
||||
method,
|
||||
result.getStatus(),
|
||||
result.getHandshakeStatus(),
|
||||
result.bytesProduced(),
|
||||
result.bytesConsumed()
|
||||
);
|
||||
}
|
||||
|
||||
private static SocketChannel createSocketChannel(final InetAddress bindAddress) throws IOException {
|
||||
final SocketChannel socketChannel = SocketChannel.open();
|
||||
if (bindAddress != null) {
|
||||
final SocketAddress socketAddress = new InetSocketAddress(bindAddress, 0);
|
||||
socketChannel.bind(socketAddress);
|
||||
}
|
||||
socketChannel.configureBlocking(false);
|
||||
return socketChannel;
|
||||
}
|
||||
|
||||
private static SSLEngine createEngine(final SSLContext sslContext, final boolean useClientMode) {
|
||||
final SSLEngine sslEngine = sslContext.createSSLEngine();
|
||||
sslEngine.setUseClientMode(useClientMode);
|
||||
sslEngine.setNeedClientAuth(CLIENT_AUTHENTICATION_REQUIRED);
|
||||
return sslEngine;
|
||||
}
|
||||
|
||||
private enum ChannelStatus {
|
||||
DISCONNECTED,
|
||||
|
||||
CONNECTING,
|
||||
|
||||
CONNECTED,
|
||||
|
||||
HANDSHAKING,
|
||||
|
||||
ESTABLISHED,
|
||||
|
||||
CLOSED
|
||||
}
|
||||
}
|
|
@ -1,61 +0,0 @@
|
|||
/*
|
||||
* 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.remote.io.socket.ssl;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
public class SSLSocketChannelInputStream extends InputStream {
|
||||
|
||||
private final SSLSocketChannel channel;
|
||||
|
||||
public SSLSocketChannelInputStream(final SSLSocketChannel channel) {
|
||||
this.channel = channel;
|
||||
}
|
||||
|
||||
public void consume() throws IOException {
|
||||
channel.consume();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
return channel.read();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(final byte[] b) throws IOException {
|
||||
return channel.read(b);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(final byte[] b, final int off, final int len) throws IOException {
|
||||
return channel.read(b, off, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the underlying SSLSocketChannel, which will also close the OutputStream and connection
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
channel.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
return channel.available();
|
||||
}
|
||||
}
|
|
@ -1,383 +0,0 @@
|
|||
/*
|
||||
* 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.remote.io.socket.ssl;
|
||||
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.bootstrap.ServerBootstrap;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelInitializer;
|
||||
import io.netty.channel.ChannelPipeline;
|
||||
import io.netty.channel.EventLoopGroup;
|
||||
import io.netty.channel.SimpleChannelInboundHandler;
|
||||
import io.netty.channel.nio.NioEventLoopGroup;
|
||||
import io.netty.channel.socket.nio.NioServerSocketChannel;
|
||||
import io.netty.channel.socket.nio.NioSocketChannel;
|
||||
import io.netty.handler.codec.DelimiterBasedFrameDecoder;
|
||||
import io.netty.handler.codec.Delimiters;
|
||||
import io.netty.handler.codec.string.StringDecoder;
|
||||
import io.netty.handler.codec.string.StringEncoder;
|
||||
import io.netty.handler.ssl.SslHandler;
|
||||
import org.apache.nifi.security.util.SslContextFactory;
|
||||
import org.apache.nifi.security.util.TemporaryKeyStoreBuilder;
|
||||
import org.apache.nifi.security.util.TlsConfiguration;
|
||||
import org.apache.nifi.security.util.TlsPlatform;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.junit.jupiter.api.condition.EnabledIf;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import javax.net.ssl.SSLException;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketAddress;
|
||||
import java.nio.channels.ServerSocketChannel;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.security.GeneralSecurityException;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@Timeout(value = 15)
|
||||
public class SSLSocketChannelTest {
|
||||
private static final String LOCALHOST = "localhost";
|
||||
|
||||
private static final int GROUP_THREADS = 1;
|
||||
|
||||
private static final boolean CLIENT_CHANNEL = true;
|
||||
|
||||
private static final boolean SERVER_CHANNEL = false;
|
||||
|
||||
private static final int CHANNEL_TIMEOUT = 15000;
|
||||
|
||||
private static final int CHANNEL_FAILURE_TIMEOUT = 100;
|
||||
|
||||
private static final int CHANNEL_POLL_TIMEOUT = 5000;
|
||||
|
||||
private static final int MAX_MESSAGE_LENGTH = 1024;
|
||||
|
||||
private static final long SHUTDOWN_TIMEOUT = 100;
|
||||
|
||||
private static final String TLS_1_3 = "TLSv1.3";
|
||||
|
||||
private static final String TLS_1_2 = "TLSv1.2";
|
||||
|
||||
private static final String MESSAGE = "PING\n";
|
||||
|
||||
private static final Charset MESSAGE_CHARSET = StandardCharsets.UTF_8;
|
||||
|
||||
private static final byte[] MESSAGE_BYTES = MESSAGE.getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
private static final int FIRST_BYTE_OFFSET = 1;
|
||||
|
||||
private static final int SINGLE_COUNT_DOWN = 1;
|
||||
|
||||
private static SSLContext sslContext;
|
||||
|
||||
private static final String TLS_1_3_SUPPORTED = "isTls13Supported";
|
||||
|
||||
public static boolean isTls13Supported() {
|
||||
return TlsPlatform.getSupportedProtocols().contains(TLS_1_3);
|
||||
}
|
||||
|
||||
@BeforeAll
|
||||
public static void setConfiguration() throws GeneralSecurityException {
|
||||
final TlsConfiguration tlsConfiguration = new TemporaryKeyStoreBuilder().build();
|
||||
sslContext = SslContextFactory.createSslContext(tlsConfiguration);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientConnectFailed() throws IOException {
|
||||
final SSLSocketChannel sslSocketChannel = new SSLSocketChannel(sslContext, "this-host-does-not-exist", 1, null, CLIENT_CHANNEL);
|
||||
sslSocketChannel.setTimeout(CHANNEL_FAILURE_TIMEOUT);
|
||||
assertThrows(Exception.class, sslSocketChannel::connect);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientConnectHandshakeFailed() throws IOException {
|
||||
final String enabledProtocol = isTls13Supported() ? TLS_1_3 : TLS_1_2;
|
||||
|
||||
final EventLoopGroup group = new NioEventLoopGroup(GROUP_THREADS);
|
||||
|
||||
try (final SocketChannel socketChannel = SocketChannel.open()) {
|
||||
final Channel serverChannel = startServer(group, enabledProtocol, getSingleCountDownLatch());
|
||||
final int port = getListeningPort(serverChannel);
|
||||
|
||||
socketChannel.connect(new InetSocketAddress(LOCALHOST, port));
|
||||
final SSLEngine sslEngine = createSslEngine(enabledProtocol, CLIENT_CHANNEL);
|
||||
|
||||
final SSLSocketChannel sslSocketChannel = new SSLSocketChannel(sslEngine, socketChannel);
|
||||
sslSocketChannel.setTimeout(CHANNEL_FAILURE_TIMEOUT);
|
||||
|
||||
shutdownGroup(group);
|
||||
assertThrows(SSLException.class, sslSocketChannel::connect);
|
||||
} finally {
|
||||
shutdownGroup(group);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientConnectWriteReadTls12() throws Exception {
|
||||
assertChannelConnectedWriteReadClosed(TLS_1_2);
|
||||
}
|
||||
|
||||
@EnabledIf(TLS_1_3_SUPPORTED)
|
||||
@Test
|
||||
public void testClientConnectWriteReadTls13() throws Exception {
|
||||
assertChannelConnectedWriteReadClosed(TLS_1_3);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientConnectWriteAvailableReadTls12() throws Exception {
|
||||
assertChannelConnectedWriteAvailableRead(TLS_1_2);
|
||||
}
|
||||
|
||||
@EnabledIf(TLS_1_3_SUPPORTED)
|
||||
@Test
|
||||
public void testClientConnectWriteAvailableReadTls13() throws Exception {
|
||||
assertChannelConnectedWriteAvailableRead(TLS_1_3);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Timeout(value = CHANNEL_TIMEOUT, unit = TimeUnit.MILLISECONDS)
|
||||
public void testServerReadWriteTls12() throws Exception {
|
||||
assertServerChannelConnectedReadClosed(TLS_1_2);
|
||||
}
|
||||
|
||||
@EnabledIf(TLS_1_3_SUPPORTED)
|
||||
@Test
|
||||
@Timeout(value = CHANNEL_TIMEOUT, unit = TimeUnit.MILLISECONDS)
|
||||
public void testServerReadWriteTls13() throws Exception {
|
||||
assertServerChannelConnectedReadClosed(TLS_1_3);
|
||||
}
|
||||
|
||||
private void assertServerChannelConnectedReadClosed(final String enabledProtocol) throws IOException, InterruptedException {
|
||||
final ServerSocketChannel serverSocketChannel = ServerSocketChannel.open();
|
||||
final SocketAddress socketAddress = new InetSocketAddress(LOCALHOST, 0);
|
||||
serverSocketChannel.bind(socketAddress);
|
||||
|
||||
final Executor executor = Executors.newSingleThreadExecutor();
|
||||
final EventLoopGroup group = new NioEventLoopGroup(GROUP_THREADS);
|
||||
try {
|
||||
final SocketAddress serverLocalAddress = serverSocketChannel.getLocalAddress();
|
||||
final int listeningPort = (serverLocalAddress instanceof InetSocketAddress) ? ((InetSocketAddress) serverLocalAddress).getPort() : 0;
|
||||
|
||||
final Channel channel = startClient(group, listeningPort, enabledProtocol);
|
||||
|
||||
try {
|
||||
final SocketChannel socketChannel = serverSocketChannel.accept();
|
||||
final SSLSocketChannel sslSocketChannel = new SSLSocketChannel(sslContext, socketChannel, SERVER_CHANNEL);
|
||||
|
||||
final BlockingQueue<String> queue = new LinkedBlockingQueue<>();
|
||||
final Runnable readCommand = () -> {
|
||||
final byte[] messageBytes = new byte[MESSAGE_BYTES.length];
|
||||
try {
|
||||
final int messageBytesRead = sslSocketChannel.read(messageBytes);
|
||||
if (messageBytesRead == MESSAGE_BYTES.length) {
|
||||
queue.add(new String(messageBytes, MESSAGE_CHARSET));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
};
|
||||
executor.execute(readCommand);
|
||||
channel.writeAndFlush(MESSAGE).syncUninterruptibly();
|
||||
|
||||
final String messageRead = queue.poll(CHANNEL_POLL_TIMEOUT, TimeUnit.MILLISECONDS);
|
||||
assertEquals(MESSAGE, messageRead, "Message not matched");
|
||||
} finally {
|
||||
channel.close();
|
||||
}
|
||||
} finally {
|
||||
shutdownGroup(group);
|
||||
serverSocketChannel.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void assertChannelConnectedWriteReadClosed(final String enabledProtocol) throws IOException {
|
||||
final CountDownLatch countDownLatch = getSingleCountDownLatch();
|
||||
processClientSslSocketChannel(enabledProtocol, countDownLatch, (sslSocketChannel -> {
|
||||
try {
|
||||
sslSocketChannel.connect();
|
||||
assertFalse(sslSocketChannel.isClosed());
|
||||
|
||||
assertChannelWriteRead(sslSocketChannel, countDownLatch);
|
||||
|
||||
sslSocketChannel.close();
|
||||
assertTrue(sslSocketChannel.isClosed());
|
||||
} catch (final IOException e) {
|
||||
throw new UncheckedIOException(String.format("Channel Failed for %s", enabledProtocol), e);
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
private void assertChannelConnectedWriteAvailableRead(final String enabledProtocol) throws IOException {
|
||||
final CountDownLatch countDownLatch = getSingleCountDownLatch();
|
||||
processClientSslSocketChannel(enabledProtocol, countDownLatch, (sslSocketChannel -> {
|
||||
try {
|
||||
sslSocketChannel.connect();
|
||||
assertFalse(sslSocketChannel.isClosed());
|
||||
|
||||
assertChannelWriteAvailableRead(sslSocketChannel, countDownLatch);
|
||||
|
||||
sslSocketChannel.close();
|
||||
assertTrue(sslSocketChannel.isClosed());
|
||||
} catch (final IOException e) {
|
||||
throw new UncheckedIOException(String.format("Channel Failed for %s", enabledProtocol), e);
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
private void assertChannelWriteAvailableRead(final SSLSocketChannel sslSocketChannel, final CountDownLatch countDownLatch) throws IOException {
|
||||
sslSocketChannel.write(MESSAGE_BYTES);
|
||||
sslSocketChannel.available();
|
||||
awaitCountDownLatch(countDownLatch);
|
||||
assetMessageRead(sslSocketChannel);
|
||||
}
|
||||
|
||||
private void assertChannelWriteRead(final SSLSocketChannel sslSocketChannel, final CountDownLatch countDownLatch) throws IOException {
|
||||
sslSocketChannel.write(MESSAGE_BYTES);
|
||||
awaitCountDownLatch(countDownLatch);
|
||||
assetMessageRead(sslSocketChannel);
|
||||
}
|
||||
|
||||
private void awaitCountDownLatch(final CountDownLatch countDownLatch) throws IOException {
|
||||
try {
|
||||
countDownLatch.await();
|
||||
} catch (final InterruptedException e) {
|
||||
throw new IOException("Count Down Interrupted", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void assetMessageRead(final SSLSocketChannel sslSocketChannel) throws IOException {
|
||||
final byte firstByteRead = (byte) sslSocketChannel.read();
|
||||
assertEquals(MESSAGE_BYTES[0], firstByteRead, "Channel Message first byte not matched");
|
||||
|
||||
final int available = sslSocketChannel.available();
|
||||
final int availableExpected = MESSAGE_BYTES.length - FIRST_BYTE_OFFSET;
|
||||
assertEquals(availableExpected, available, "Available Bytes not matched");
|
||||
|
||||
final byte[] messageBytes = new byte[MESSAGE_BYTES.length];
|
||||
messageBytes[0] = firstByteRead;
|
||||
|
||||
final int messageBytesRead = sslSocketChannel.read(messageBytes, FIRST_BYTE_OFFSET, messageBytes.length);
|
||||
assertEquals(messageBytes.length - FIRST_BYTE_OFFSET, messageBytesRead, "Channel Message Bytes Read not matched");
|
||||
|
||||
final String message = new String(messageBytes, MESSAGE_CHARSET);
|
||||
assertEquals(MESSAGE, message, "Message not matched");
|
||||
}
|
||||
|
||||
private void processClientSslSocketChannel(final String enabledProtocol, final CountDownLatch countDownLatch, final Consumer<SSLSocketChannel> channelConsumer) throws IOException {
|
||||
final EventLoopGroup group = new NioEventLoopGroup(GROUP_THREADS);
|
||||
|
||||
try {
|
||||
final Channel channel = startServer(group, enabledProtocol, countDownLatch);
|
||||
final int port = getListeningPort(channel);
|
||||
final SSLSocketChannel sslSocketChannel = new SSLSocketChannel(sslContext, LOCALHOST, port, null, CLIENT_CHANNEL);
|
||||
sslSocketChannel.setTimeout(CHANNEL_TIMEOUT);
|
||||
channelConsumer.accept(sslSocketChannel);
|
||||
} finally {
|
||||
shutdownGroup(group);
|
||||
}
|
||||
}
|
||||
|
||||
private int getListeningPort(final Channel serverChannel) {
|
||||
final SocketAddress address = serverChannel.localAddress();
|
||||
if (address instanceof InetSocketAddress) {
|
||||
return ((InetSocketAddress) address).getPort();
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
private Channel startClient(final EventLoopGroup group, final int port, final String enabledProtocol) {
|
||||
final Bootstrap bootstrap = new Bootstrap();
|
||||
bootstrap.group(group);
|
||||
bootstrap.channel(NioSocketChannel.class);
|
||||
bootstrap.handler(new ChannelInitializer<Channel>() {
|
||||
@Override
|
||||
protected void initChannel(final Channel channel) {
|
||||
final ChannelPipeline pipeline = channel.pipeline();
|
||||
final SSLEngine sslEngine = createSslEngine(enabledProtocol, CLIENT_CHANNEL);
|
||||
setPipelineHandlers(pipeline, sslEngine);
|
||||
}
|
||||
});
|
||||
return bootstrap.connect(LOCALHOST, port).syncUninterruptibly().channel();
|
||||
}
|
||||
|
||||
private Channel startServer(final EventLoopGroup group, final String enabledProtocol, final CountDownLatch countDownLatch) {
|
||||
final ServerBootstrap bootstrap = new ServerBootstrap();
|
||||
bootstrap.group(group);
|
||||
bootstrap.channel(NioServerSocketChannel.class);
|
||||
bootstrap.childHandler(new ChannelInitializer<Channel>() {
|
||||
@Override
|
||||
protected void initChannel(final Channel channel) {
|
||||
final ChannelPipeline pipeline = channel.pipeline();
|
||||
final SSLEngine sslEngine = createSslEngine(enabledProtocol, SERVER_CHANNEL);
|
||||
setPipelineHandlers(pipeline, sslEngine);
|
||||
pipeline.addLast(new SimpleChannelInboundHandler<String>() {
|
||||
@Override
|
||||
protected void channelRead0(ChannelHandlerContext channelHandlerContext, String s) throws Exception {
|
||||
channelHandlerContext.channel().writeAndFlush(MESSAGE).sync();
|
||||
countDownLatch.countDown();
|
||||
}
|
||||
});
|
||||
}
|
||||
});
|
||||
|
||||
final ChannelFuture bindFuture = bootstrap.bind(LOCALHOST, 0);
|
||||
bindFuture.syncUninterruptibly();
|
||||
return bindFuture.channel();
|
||||
}
|
||||
|
||||
private SSLEngine createSslEngine(final String enabledProtocol, final boolean useClientMode) {
|
||||
final SSLEngine sslEngine = sslContext.createSSLEngine();
|
||||
sslEngine.setUseClientMode(useClientMode);
|
||||
sslEngine.setEnabledProtocols(new String[]{enabledProtocol});
|
||||
return sslEngine;
|
||||
}
|
||||
|
||||
private void setPipelineHandlers(final ChannelPipeline pipeline, final SSLEngine sslEngine) {
|
||||
pipeline.addLast(new SslHandler(sslEngine));
|
||||
pipeline.addLast(new DelimiterBasedFrameDecoder(MAX_MESSAGE_LENGTH, Delimiters.lineDelimiter()));
|
||||
pipeline.addLast(new StringDecoder());
|
||||
pipeline.addLast(new StringEncoder());
|
||||
}
|
||||
|
||||
private void shutdownGroup(final EventLoopGroup group) {
|
||||
group.shutdownGracefully(SHUTDOWN_TIMEOUT, SHUTDOWN_TIMEOUT, TimeUnit.MILLISECONDS).syncUninterruptibly();
|
||||
}
|
||||
|
||||
private CountDownLatch getSingleCountDownLatch() {
|
||||
return new CountDownLatch(SINGLE_COUNT_DOWN);
|
||||
}
|
||||
}
|
|
@ -50,7 +50,6 @@
|
|||
<module>nifi-security-identity</module>
|
||||
<module>nifi-security-kerberos-api</module>
|
||||
<module>nifi-security-kerberos</module>
|
||||
<module>nifi-security-socket-ssl</module>
|
||||
<module>nifi-security-ssl</module>
|
||||
<module>nifi-security-utils-api</module>
|
||||
<module>nifi-security-utils</module>
|
||||
|
|
|
@ -2637,10 +2637,10 @@ Sometimes it may be desirable to deprecate a component. Whenever this occurs the
|
|||
----
|
||||
@DeprecationNotice(
|
||||
reason = "Legacy Protocol has been superseded",
|
||||
alternatives = {ListenSyslog.class},
|
||||
classNames = {"org.apache.nifi.processors.standard.ListenRELP"}
|
||||
alternatives = {SomeNewProcessor.class},
|
||||
classNames = {"org.apache.nifi.processors.standard.SomeOldProcessor"}
|
||||
)
|
||||
public class ListenLegacyProtocol extends AbstractProcessor {}
|
||||
public class SomeOldProcessor extends AbstractProcessor {}
|
||||
----
|
||||
The `alternatives` property can be used to define an array of recommended replacement Components, while `classNames`
|
||||
can be used to represent similar content through an array of class name strings.
|
||||
|
@ -2649,7 +2649,7 @@ Adding the `@DeprecationNotice` annotation renders a warning message in generate
|
|||
following warning when the Flow Configuration includes the component:
|
||||
|
||||
----
|
||||
Added Deprecated Component ListenLegacyProtocol[id=929a52c7-1e3e-423e-b303-6ca2ef657617] See alternatives [ListenSyslog,ListenRELP]
|
||||
Added Deprecated Component SomeOldProcessor[id=929a52c7-1e3e-423e-b303-6ca2ef657617] See alternatives [SomeNewProcessor]
|
||||
----
|
||||
|
||||
=== Feature Deprecation
|
||||
|
|
|
@ -333,7 +333,6 @@ categorizing them by their functions.
|
|||
is valid according to the user-defined XML Schema.
|
||||
|
||||
=== Database Access
|
||||
- *ConvertJSONToSQL*: Convert a JSON document into a SQL INSERT or UPDATE command that can then be passed to the PutSQL Processor
|
||||
- *ExecuteSQL*: Executes a user-defined SQL SELECT command, writing the results to a FlowFile in Avro format
|
||||
- *PutSQL*: Updates a database by executing the SQL DDM statement defined by the FlowFile's content
|
||||
- *SelectHiveQL*: Executes a user-defined HiveQL SELECT command against an Apache Hive database, writing the results to a FlowFile in Avro or CSV format
|
||||
|
|
|
@ -24,12 +24,6 @@
|
|||
<artifactId>nifi-standard-record-utils</artifactId>
|
||||
|
||||
<dependencies>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-socket-ssl</artifactId>
|
||||
<version>2.0.0-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-utils-api</artifactId>
|
||||
|
|
|
@ -1,36 +0,0 @@
|
|||
/*
|
||||
* 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.record.listen;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
public class IOUtils {
|
||||
|
||||
public static void closeQuietly(final Closeable closeable) {
|
||||
if (closeable == null) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
|
||||
closeable.close();
|
||||
} catch (final IOException ioe) {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,98 +0,0 @@
|
|||
/*
|
||||
* 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.record.listen;
|
||||
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelInputStream;
|
||||
import org.apache.nifi.schema.access.SchemaNotFoundException;
|
||||
import org.apache.nifi.serialization.MalformedRecordException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import javax.net.ssl.SSLSession;
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.InetAddress;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Encapsulates an SSLSocketChannel and a RecordReader created for the given channel.
|
||||
*/
|
||||
public class SSLSocketChannelRecordReader implements SocketChannelRecordReader {
|
||||
|
||||
private final SocketChannel socketChannel;
|
||||
private final SSLSocketChannel sslSocketChannel;
|
||||
private final RecordReaderFactory readerFactory;
|
||||
private final SocketChannelRecordReaderDispatcher dispatcher;
|
||||
private final SSLEngine sslEngine;
|
||||
|
||||
private RecordReader recordReader;
|
||||
|
||||
public SSLSocketChannelRecordReader(final SocketChannel socketChannel,
|
||||
final SSLSocketChannel sslSocketChannel,
|
||||
final RecordReaderFactory readerFactory,
|
||||
final SocketChannelRecordReaderDispatcher dispatcher,
|
||||
final SSLEngine sslEngine) {
|
||||
this.socketChannel = socketChannel;
|
||||
this.sslSocketChannel = sslSocketChannel;
|
||||
this.readerFactory = readerFactory;
|
||||
this.dispatcher = dispatcher;
|
||||
this.sslEngine = sslEngine;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader createRecordReader(final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException {
|
||||
if (recordReader != null) {
|
||||
throw new IllegalStateException("Cannot create RecordReader because already created");
|
||||
}
|
||||
|
||||
final InputStream socketIn = new SSLSocketChannelInputStream(sslSocketChannel);
|
||||
final InputStream in = new BufferedInputStream(socketIn);
|
||||
recordReader = readerFactory.createRecordReader(Collections.emptyMap(), in, -1, logger);
|
||||
return recordReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader getRecordReader() {
|
||||
return recordReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InetAddress getRemoteAddress() {
|
||||
return socketChannel.socket().getInetAddress();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return sslSocketChannel.isClosed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
IOUtils.closeQuietly(recordReader);
|
||||
IOUtils.closeQuietly(sslSocketChannel);
|
||||
dispatcher.connectionCompleted();
|
||||
}
|
||||
|
||||
public SSLSession getSession() {
|
||||
return sslEngine.getSession();
|
||||
}
|
||||
}
|
|
@ -1,58 +0,0 @@
|
|||
/*
|
||||
* 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.record.listen;
|
||||
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.schema.access.SchemaNotFoundException;
|
||||
import org.apache.nifi.serialization.MalformedRecordException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
|
||||
/**
|
||||
* Encapsulates a SocketChannel and a RecordReader for the channel.
|
||||
*/
|
||||
public interface SocketChannelRecordReader extends Closeable {
|
||||
|
||||
/**
|
||||
* Lazily creates the RecordReader.
|
||||
*
|
||||
* @param logger the logger of the component creating the reader
|
||||
* @return a RecordReader
|
||||
*
|
||||
* @throws IllegalStateException if create is called after a reader has already been created
|
||||
*/
|
||||
RecordReader createRecordReader(ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException;
|
||||
|
||||
/**
|
||||
* @return the RecordReader created by calling createRecordReader, or null if one has not been created yet
|
||||
*/
|
||||
RecordReader getRecordReader();
|
||||
|
||||
/**
|
||||
* @return the remote address of the underlying channel
|
||||
*/
|
||||
InetAddress getRemoteAddress();
|
||||
|
||||
/**
|
||||
* @return true if the underlying channel is closed, false otherwise
|
||||
*/
|
||||
boolean isClosed();
|
||||
|
||||
}
|
|
@ -1,148 +0,0 @@
|
|||
/*
|
||||
* 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.record.listen;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.net.SocketAddress;
|
||||
import java.net.StandardSocketOptions;
|
||||
import java.nio.channels.ServerSocketChannel;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
import org.apache.nifi.security.util.ClientAuth;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
|
||||
/**
|
||||
* Accepts connections on the given ServerSocketChannel and dispatches a SocketChannelRecordReader for processing.
|
||||
*/
|
||||
public class SocketChannelRecordReaderDispatcher implements Runnable, Closeable {
|
||||
|
||||
private final ServerSocketChannel serverSocketChannel;
|
||||
private final SSLContext sslContext;
|
||||
private final ClientAuth clientAuth;
|
||||
private final int socketReadTimeout;
|
||||
private final int receiveBufferSize;
|
||||
private final int maxConnections;
|
||||
private final RecordReaderFactory readerFactory;
|
||||
private final BlockingQueue<SocketChannelRecordReader> recordReaders;
|
||||
private final ComponentLog logger;
|
||||
|
||||
private final AtomicInteger currentConnections = new AtomicInteger(0);
|
||||
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
public SocketChannelRecordReaderDispatcher(final ServerSocketChannel serverSocketChannel,
|
||||
final SSLContext sslContext,
|
||||
final ClientAuth clientAuth,
|
||||
final int socketReadTimeout,
|
||||
final int receiveBufferSize,
|
||||
final int maxConnections,
|
||||
final RecordReaderFactory readerFactory,
|
||||
final BlockingQueue<SocketChannelRecordReader> recordReaders,
|
||||
final ComponentLog logger) {
|
||||
this.serverSocketChannel = serverSocketChannel;
|
||||
this.sslContext = sslContext;
|
||||
this.clientAuth = clientAuth;
|
||||
this.socketReadTimeout = socketReadTimeout;
|
||||
this.receiveBufferSize = receiveBufferSize;
|
||||
this.maxConnections = maxConnections;
|
||||
this.readerFactory = readerFactory;
|
||||
this.recordReaders = recordReaders;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (!stopped) {
|
||||
try {
|
||||
final SocketChannel socketChannel = serverSocketChannel.accept();
|
||||
if (socketChannel == null) {
|
||||
Thread.sleep(20);
|
||||
continue;
|
||||
}
|
||||
|
||||
socketChannel.setOption(StandardSocketOptions.SO_KEEPALIVE, true);
|
||||
final SocketAddress remoteSocketAddress = socketChannel.getRemoteAddress();
|
||||
socketChannel.socket().setSoTimeout(socketReadTimeout);
|
||||
socketChannel.socket().setReceiveBufferSize(receiveBufferSize);
|
||||
|
||||
if (currentConnections.incrementAndGet() > maxConnections) {
|
||||
currentConnections.decrementAndGet();
|
||||
final String remoteAddress = remoteSocketAddress == null ? "null" : remoteSocketAddress.toString();
|
||||
logger.warn("Rejecting connection from {} because max connections has been met", remoteAddress);
|
||||
IOUtils.closeQuietly(socketChannel);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (logger.isDebugEnabled()) {
|
||||
final String remoteAddress = remoteSocketAddress == null ? "null" : remoteSocketAddress.toString();
|
||||
logger.debug("Accepted connection from {}", remoteAddress);
|
||||
}
|
||||
|
||||
// create a StandardSocketChannelRecordReader or an SSLSocketChannelRecordReader based on presence of SSLContext
|
||||
final SocketChannelRecordReader socketChannelRecordReader;
|
||||
if (sslContext == null) {
|
||||
socketChannelRecordReader = new StandardSocketChannelRecordReader(socketChannel, readerFactory, this);
|
||||
} else {
|
||||
final SSLEngine sslEngine = sslContext.createSSLEngine();
|
||||
sslEngine.setUseClientMode(false);
|
||||
|
||||
switch (clientAuth) {
|
||||
case REQUIRED:
|
||||
sslEngine.setNeedClientAuth(true);
|
||||
break;
|
||||
case WANT:
|
||||
sslEngine.setWantClientAuth(true);
|
||||
break;
|
||||
case NONE:
|
||||
sslEngine.setNeedClientAuth(false);
|
||||
sslEngine.setWantClientAuth(false);
|
||||
break;
|
||||
}
|
||||
|
||||
final SSLSocketChannel sslSocketChannel = new SSLSocketChannel(sslEngine, socketChannel);
|
||||
socketChannelRecordReader = new SSLSocketChannelRecordReader(socketChannel, sslSocketChannel, readerFactory, this, sslEngine);
|
||||
}
|
||||
|
||||
// queue the SocketChannelRecordReader for processing by the processor
|
||||
recordReaders.offer(socketChannelRecordReader);
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.error("Error dispatching connection", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public int getPort() {
|
||||
return serverSocketChannel == null ? 0 : serverSocketChannel.socket().getLocalPort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
this.stopped = true;
|
||||
IOUtils.closeQuietly(this.serverSocketChannel);
|
||||
}
|
||||
|
||||
public void connectionCompleted() {
|
||||
currentConnections.decrementAndGet();
|
||||
}
|
||||
|
||||
}
|
|
@ -1,84 +0,0 @@
|
|||
/*
|
||||
* 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.record.listen;
|
||||
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.schema.access.SchemaNotFoundException;
|
||||
import org.apache.nifi.serialization.MalformedRecordException;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.InetAddress;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Encapsulates a SocketChannel and a RecordReader created for the given channel.
|
||||
*/
|
||||
public class StandardSocketChannelRecordReader implements SocketChannelRecordReader {
|
||||
|
||||
private final SocketChannel socketChannel;
|
||||
private final RecordReaderFactory readerFactory;
|
||||
private final SocketChannelRecordReaderDispatcher dispatcher;
|
||||
|
||||
private RecordReader recordReader;
|
||||
|
||||
public StandardSocketChannelRecordReader(final SocketChannel socketChannel,
|
||||
final RecordReaderFactory readerFactory,
|
||||
final SocketChannelRecordReaderDispatcher dispatcher) {
|
||||
this.socketChannel = socketChannel;
|
||||
this.readerFactory = readerFactory;
|
||||
this.dispatcher = dispatcher;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader createRecordReader(final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException {
|
||||
if (recordReader != null) {
|
||||
throw new IllegalStateException("Cannot create RecordReader because already created");
|
||||
}
|
||||
|
||||
final InputStream socketIn = socketChannel.socket().getInputStream();
|
||||
final InputStream in = new BufferedInputStream(socketIn);
|
||||
recordReader = readerFactory.createRecordReader(Collections.emptyMap(), in, -1, logger);
|
||||
return recordReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RecordReader getRecordReader() {
|
||||
return recordReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InetAddress getRemoteAddress() {
|
||||
return socketChannel.socket().getInetAddress();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return !socketChannel.isOpen();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
IOUtils.closeQuietly(recordReader);
|
||||
IOUtils.closeQuietly(socketChannel);
|
||||
dispatcher.connectionCompleted();
|
||||
}
|
||||
}
|
|
@ -616,14 +616,6 @@
|
|||
<exclude>src/test/resources/TestConversions/data.int_float_string.without_schema.avro</exclude>
|
||||
<exclude>src/test/resources/TestConversions/data.int_float_string.xml</exclude>
|
||||
<exclude>src/test/resources/TestConversions/explicit.schema.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/malformed-person-extra-comma.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/person-1.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/person-with-bool.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/person-with-extra-field.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/person-with-null-code.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/person-without-code.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/person-without-id.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertJSONToSQL/persons.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertRecord/input/person_bad_enum.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertRecord/input/person_long_id.json</exclude>
|
||||
<exclude>src/test/resources/TestConvertRecord/input/person.json</exclude>
|
||||
|
|
|
@ -1,989 +0,0 @@
|
|||
/*
|
||||
* 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.standard;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.node.ArrayNode;
|
||||
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
||||
import com.github.benmanes.caffeine.cache.Cache;
|
||||
import com.github.benmanes.caffeine.cache.Caffeine;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
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.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.dbcp.DBCPService;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DatabaseMetaData;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Types;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
|
||||
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
|
||||
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
|
||||
import static org.apache.nifi.flowfile.attributes.FragmentAttributes.copyAttributesToOriginal;
|
||||
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@SeeAlso({PutSQL.class, PutDatabaseRecord.class})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"json", "sql", "database", "rdbms", "insert", "update", "delete", "relational", "flat"})
|
||||
@CapabilityDescription("Converts a JSON-formatted FlowFile into an UPDATE, INSERT, or DELETE SQL statement. The incoming FlowFile is expected to be "
|
||||
+ "\"flat\" JSON message, meaning that it consists of a single JSON element and each field maps to a simple type. If a field maps to "
|
||||
+ "a JSON object, that JSON object will be interpreted as Text. If the input is an array of JSON elements, each element in the array is "
|
||||
+ "output as a separate FlowFile to the 'sql' relationship. Upon successful conversion, the original FlowFile is routed to the 'original' "
|
||||
+ "relationship and the SQL is routed to the 'sql' relationship.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "mime.type", description = "Sets mime.type of FlowFile that is routed to 'sql' to 'text/plain'."),
|
||||
@WritesAttribute(attribute = "<sql>.table", description = "Sets the <sql>.table attribute of FlowFile that is routed to 'sql' to the name of the table that is updated by the SQL statement. "
|
||||
+ "The prefix for this attribute ('sql', e.g.) is determined by the SQL Parameter Attribute Prefix property."),
|
||||
@WritesAttribute(attribute = "<sql>.catalog", description = "If the Catalog name is set for this database, specifies the name of the catalog that the SQL statement will update. "
|
||||
+ "If no catalog is used, this attribute will not be added. The prefix for this attribute ('sql', e.g.) is determined by the SQL Parameter Attribute Prefix property."),
|
||||
@WritesAttribute(attribute = "fragment.identifier", description = "All FlowFiles routed to the 'sql' relationship for the same incoming FlowFile (multiple will be output for "
|
||||
+ "the same incoming FlowFile if the incoming FlowFile is a JSON Array) will have the same value for the fragment.identifier attribute. This can then be used to "
|
||||
+ "correlate the results."),
|
||||
@WritesAttribute(attribute = "fragment.count", description = "The number of SQL FlowFiles that were produced for same incoming FlowFile. This can be used in conjunction with the "
|
||||
+ "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming FlowFile."),
|
||||
@WritesAttribute(attribute = "fragment.index", description = "The position of this FlowFile in the list of outgoing FlowFiles that were all derived from the same incoming FlowFile. "
|
||||
+ "This can be used in conjunction with the fragment.identifier and fragment.count attributes to know which FlowFiles originated from the same incoming FlowFile and in what "
|
||||
+ "order the SQL FlowFiles were produced"),
|
||||
@WritesAttribute(attribute = "<sql>.args.N.type", description = "The output SQL statements are parametrized in order to avoid SQL Injection Attacks. The types of the Parameters "
|
||||
+ "to use are stored in attributes named <sql>.args.1.type, <sql>.args.2.type, <sql>.args.3.type, and so on. The type is a number representing a JDBC Type constant. "
|
||||
+ "Generally, this is useful only for software to read and interpret but is added so that a processor such as PutSQL can understand how to interpret the values. "
|
||||
+ "The prefix for this attribute ('sql', e.g.) is determined by the SQL Parameter Attribute Prefix property."),
|
||||
@WritesAttribute(attribute = "<sql>.args.N.value", description = "The output SQL statements are parametrized in order to avoid SQL Injection Attacks. The values of the Parameters "
|
||||
+ "to use are stored in the attributes named sql.args.1.value, sql.args.2.value, sql.args.3.value, and so on. Each of these attributes has a corresponding "
|
||||
+ "<sql>.args.N.type attribute that indicates how the value should be interpreted when inserting it into the database."
|
||||
+ "The prefix for this attribute ('sql', e.g.) is determined by the SQL Parameter Attribute Prefix property.")
|
||||
})
|
||||
public class ConvertJSONToSQL extends AbstractProcessor {
|
||||
private static final String UPDATE_TYPE = "UPDATE";
|
||||
private static final String INSERT_TYPE = "INSERT";
|
||||
private static final String DELETE_TYPE = "DELETE";
|
||||
static final String USE_ATTR_TYPE = "Use statement.type Attribute";
|
||||
|
||||
static final String STATEMENT_TYPE_ATTRIBUTE = "statement.type";
|
||||
|
||||
static final AllowableValue IGNORE_UNMATCHED_FIELD = new AllowableValue("Ignore Unmatched Fields", "Ignore Unmatched Fields",
|
||||
"Any field in the JSON document that cannot be mapped to a column in the database is ignored");
|
||||
static final AllowableValue FAIL_UNMATCHED_FIELD = new AllowableValue("Fail", "Fail",
|
||||
"If the JSON document has any field that cannot be mapped to a column in the database, the FlowFile will be routed to the failure relationship");
|
||||
static final AllowableValue IGNORE_UNMATCHED_COLUMN = new AllowableValue("Ignore Unmatched Columns",
|
||||
"Ignore Unmatched Columns",
|
||||
"Any column in the database that does not have a field in the JSON document will be assumed to not be required. No notification will be logged");
|
||||
static final AllowableValue WARNING_UNMATCHED_COLUMN = new AllowableValue("Warn on Unmatched Columns",
|
||||
"Warn on Unmatched Columns",
|
||||
"Any column in the database that does not have a field in the JSON document will be assumed to not be required. A warning will be logged");
|
||||
static final AllowableValue FAIL_UNMATCHED_COLUMN = new AllowableValue("Fail on Unmatched Columns",
|
||||
"Fail on Unmatched Columns",
|
||||
"A flow will fail if any column in the database that does not have a field in the JSON document. An error will be logged");
|
||||
|
||||
static final PropertyDescriptor CONNECTION_POOL = new PropertyDescriptor.Builder()
|
||||
.name("JDBC Connection Pool")
|
||||
.description("Specifies the JDBC Connection Pool to use in order to convert the JSON message to a SQL statement. "
|
||||
+ "The Connection Pool is necessary in order to determine the appropriate database column types.")
|
||||
.identifiesControllerService(DBCPService.class)
|
||||
.required(true)
|
||||
.build();
|
||||
static final PropertyDescriptor STATEMENT_TYPE = new PropertyDescriptor.Builder()
|
||||
.name("Statement Type")
|
||||
.description("Specifies the type of SQL Statement to generate")
|
||||
.required(true)
|
||||
.allowableValues(UPDATE_TYPE, INSERT_TYPE, DELETE_TYPE, USE_ATTR_TYPE)
|
||||
.build();
|
||||
static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
|
||||
.name("Table Name")
|
||||
.description("The name of the table that the statement should update")
|
||||
.required(true)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
static final PropertyDescriptor CATALOG_NAME = new PropertyDescriptor.Builder()
|
||||
.name("Catalog Name")
|
||||
.description("The name of the catalog that the statement should update. This may not apply for the database that you are updating. In this case, leave the field empty")
|
||||
.required(false)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
static final PropertyDescriptor SCHEMA_NAME = new PropertyDescriptor.Builder()
|
||||
.name("Schema Name")
|
||||
.description("The name of the schema that the table belongs to. This may not apply for the database that you are updating. In this case, leave the field empty")
|
||||
.required(false)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
static final PropertyDescriptor TRANSLATE_FIELD_NAMES = new PropertyDescriptor.Builder()
|
||||
.name("Translate Field Names")
|
||||
.description("If true, the Processor will attempt to translate JSON field names into the appropriate column names for the table specified. "
|
||||
+ "If false, the JSON field names must match the column names exactly, or the column will not be updated")
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("true")
|
||||
.build();
|
||||
static final PropertyDescriptor UNMATCHED_FIELD_BEHAVIOR = new PropertyDescriptor.Builder()
|
||||
.name("Unmatched Field Behavior")
|
||||
.description("If an incoming JSON element has a field that does not map to any of the database table's columns, this property specifies how to handle the situation")
|
||||
.allowableValues(IGNORE_UNMATCHED_FIELD, FAIL_UNMATCHED_FIELD)
|
||||
.defaultValue(IGNORE_UNMATCHED_FIELD.getValue())
|
||||
.build();
|
||||
static final PropertyDescriptor UNMATCHED_COLUMN_BEHAVIOR = new PropertyDescriptor.Builder()
|
||||
.name("Unmatched Column Behavior")
|
||||
.description("If an incoming JSON element does not have a field mapping for all of the database table's columns, this property specifies how to handle the situation")
|
||||
.allowableValues(IGNORE_UNMATCHED_COLUMN, WARNING_UNMATCHED_COLUMN, FAIL_UNMATCHED_COLUMN)
|
||||
.defaultValue(FAIL_UNMATCHED_COLUMN.getValue())
|
||||
.build();
|
||||
static final PropertyDescriptor UPDATE_KEY = new PropertyDescriptor.Builder()
|
||||
.name("Update Keys")
|
||||
.description("A comma-separated list of column names that uniquely identifies a row in the database for UPDATE statements. "
|
||||
+ "If the Statement Type is UPDATE and this property is not set, the table's Primary Keys are used. "
|
||||
+ "In this case, if no Primary Key exists, the conversion to SQL will fail if Unmatched Column Behaviour is set to FAIL. "
|
||||
+ "This property is ignored if the Statement Type is INSERT")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.required(false)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor QUOTED_IDENTIFIERS = new PropertyDescriptor.Builder()
|
||||
.name("jts-quoted-identifiers")
|
||||
.displayName("Quote Column Identifiers")
|
||||
.description("Enabling this option will cause all column names to be quoted, allowing you to "
|
||||
+ "use reserved words as column names in your tables.")
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("false")
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor QUOTED_TABLE_IDENTIFIER = new PropertyDescriptor.Builder()
|
||||
.name("jts-quoted-table-identifiers")
|
||||
.displayName("Quote Table Identifiers")
|
||||
.description("Enabling this option will cause the table name to be quoted to support the "
|
||||
+ "use of special characters in the table name")
|
||||
.allowableValues("true", "false")
|
||||
.defaultValue("false")
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor SQL_PARAM_ATTR_PREFIX = new PropertyDescriptor.Builder()
|
||||
.name("jts-sql-param-attr-prefix")
|
||||
.displayName("SQL Parameter Attribute Prefix")
|
||||
.description("The string to be prepended to the outgoing flow file attributes, such as <sql>.args.1.value, where <sql> is replaced with the specified value")
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
|
||||
.addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
|
||||
.required(true)
|
||||
.defaultValue("sql")
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor TABLE_SCHEMA_CACHE_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("table-schema-cache-size")
|
||||
.displayName("Table Schema Cache Size")
|
||||
.description("Specifies how many Table Schemas should be cached")
|
||||
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
||||
.defaultValue("100")
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final Relationship REL_ORIGINAL = new Relationship.Builder()
|
||||
.name("original")
|
||||
.description("When a FlowFile is converted to SQL, the original JSON FlowFile is routed to this relationship")
|
||||
.build();
|
||||
static final Relationship REL_SQL = new Relationship.Builder()
|
||||
.name("sql")
|
||||
.description("A FlowFile is routed to this relationship when its contents have successfully been converted into a SQL statement")
|
||||
.build();
|
||||
static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("A FlowFile is routed to this relationship if it cannot be converted into a SQL statement. Common causes include invalid JSON "
|
||||
+ "content or the JSON content missing a required field (if using an INSERT statement type).")
|
||||
.build();
|
||||
|
||||
private Cache<SchemaKey, TableSchema> schemaCache;
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
final List<PropertyDescriptor> properties = new ArrayList<>();
|
||||
properties.add(CONNECTION_POOL);
|
||||
properties.add(STATEMENT_TYPE);
|
||||
properties.add(TABLE_NAME);
|
||||
properties.add(CATALOG_NAME);
|
||||
properties.add(SCHEMA_NAME);
|
||||
properties.add(TRANSLATE_FIELD_NAMES);
|
||||
properties.add(UNMATCHED_FIELD_BEHAVIOR);
|
||||
properties.add(UNMATCHED_COLUMN_BEHAVIOR);
|
||||
properties.add(UPDATE_KEY);
|
||||
properties.add(QUOTED_IDENTIFIERS);
|
||||
properties.add(QUOTED_TABLE_IDENTIFIER);
|
||||
properties.add(SQL_PARAM_ATTR_PREFIX);
|
||||
properties.add(TABLE_SCHEMA_CACHE_SIZE);
|
||||
return properties;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
final Set<Relationship> rels = new HashSet<>();
|
||||
rels.add(REL_ORIGINAL);
|
||||
rels.add(REL_SQL);
|
||||
rels.add(REL_FAILURE);
|
||||
return rels;
|
||||
}
|
||||
|
||||
|
||||
@OnScheduled
|
||||
public void onScheduled(final ProcessContext context) {
|
||||
final int tableSchemaCacheSize = context.getProperty(TABLE_SCHEMA_CACHE_SIZE).asInteger();
|
||||
schemaCache = Caffeine.newBuilder()
|
||||
.maximumSize(tableSchemaCacheSize)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
final FlowFile flowFile = session.get();
|
||||
if (flowFile == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final boolean translateFieldNames = context.getProperty(TRANSLATE_FIELD_NAMES).asBoolean();
|
||||
final boolean ignoreUnmappedFields = IGNORE_UNMATCHED_FIELD.getValue().equalsIgnoreCase(context.getProperty(UNMATCHED_FIELD_BEHAVIOR).getValue());
|
||||
String statementType = context.getProperty(STATEMENT_TYPE).getValue();
|
||||
if (USE_ATTR_TYPE.equals(statementType)) {
|
||||
statementType = flowFile.getAttribute(STATEMENT_TYPE_ATTRIBUTE);
|
||||
}
|
||||
final String updateKeys = context.getProperty(UPDATE_KEY).evaluateAttributeExpressions(flowFile).getValue();
|
||||
|
||||
final String catalog = context.getProperty(CATALOG_NAME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final String schemaName = context.getProperty(SCHEMA_NAME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final SchemaKey schemaKey = new SchemaKey(catalog, tableName);
|
||||
final boolean includePrimaryKeys = UPDATE_TYPE.equals(statementType) && updateKeys == null;
|
||||
|
||||
// Is the unmatched column behaviour fail or warning?
|
||||
final boolean failUnmappedColumns = FAIL_UNMATCHED_COLUMN.getValue().equalsIgnoreCase(context.getProperty(UNMATCHED_COLUMN_BEHAVIOR).getValue());
|
||||
final boolean warningUnmappedColumns = WARNING_UNMATCHED_COLUMN.getValue().equalsIgnoreCase(context.getProperty(UNMATCHED_COLUMN_BEHAVIOR).getValue());
|
||||
|
||||
//Escape column names?
|
||||
final boolean escapeColumnNames = context.getProperty(QUOTED_IDENTIFIERS).asBoolean();
|
||||
|
||||
// Quote table name?
|
||||
final boolean quoteTableName = context.getProperty(QUOTED_TABLE_IDENTIFIER).asBoolean();
|
||||
|
||||
// Attribute prefix
|
||||
final String attributePrefix = context.getProperty(SQL_PARAM_ATTR_PREFIX).evaluateAttributeExpressions(flowFile).getValue();
|
||||
|
||||
TableSchema schema;
|
||||
try {
|
||||
schema = schemaCache.get(schemaKey, key -> {
|
||||
final DBCPService dbcpService = context.getProperty(CONNECTION_POOL).asControllerService(DBCPService.class);
|
||||
try (final Connection conn = dbcpService.getConnection(flowFile.getAttributes())) {
|
||||
return TableSchema.from(conn, catalog, schemaName, tableName, translateFieldNames, includePrimaryKeys);
|
||||
} catch (final SQLException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
});
|
||||
} catch (ProcessException e) {
|
||||
getLogger().error("Failed to convert {} into a SQL statement", flowFile, e);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
return;
|
||||
}
|
||||
|
||||
// Parse the JSON document
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final AtomicReference<JsonNode> rootNodeRef = new AtomicReference<>(null);
|
||||
try {
|
||||
session.read(flowFile, new InputStreamCallback() {
|
||||
@Override
|
||||
public void process(final InputStream in) throws IOException {
|
||||
try (final InputStream bufferedIn = new BufferedInputStream(in)) {
|
||||
rootNodeRef.set(mapper.readTree(bufferedIn));
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (final ProcessException pe) {
|
||||
getLogger().error("Failed to parse {} as JSON", flowFile, pe);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
return;
|
||||
}
|
||||
|
||||
final JsonNode rootNode = rootNodeRef.get();
|
||||
|
||||
// The node may or may not be a Json Array. If it isn't, we will create an
|
||||
// ArrayNode and add just the root node to it. We do this so that we can easily iterate
|
||||
// over the array node, rather than duplicating the logic or creating another function that takes many variables
|
||||
// in order to implement the logic.
|
||||
final ArrayNode arrayNode;
|
||||
if (rootNode.isArray()) {
|
||||
arrayNode = (ArrayNode) rootNode;
|
||||
} else {
|
||||
final JsonNodeFactory nodeFactory = JsonNodeFactory.instance;
|
||||
arrayNode = new ArrayNode(nodeFactory);
|
||||
arrayNode.add(rootNode);
|
||||
}
|
||||
|
||||
final String fragmentIdentifier = UUID.randomUUID().toString();
|
||||
|
||||
final Set<FlowFile> created = new HashSet<>();
|
||||
for (int i = 0; i < arrayNode.size(); i++) {
|
||||
final JsonNode jsonNode = arrayNode.get(i);
|
||||
|
||||
final String sql;
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
||||
try {
|
||||
// build the fully qualified table name
|
||||
final StringBuilder tableNameBuilder = new StringBuilder();
|
||||
if (catalog != null) {
|
||||
tableNameBuilder.append(catalog).append(".");
|
||||
}
|
||||
if (schemaName != null) {
|
||||
tableNameBuilder.append(schemaName).append(".");
|
||||
}
|
||||
tableNameBuilder.append(tableName);
|
||||
final String fqTableName = tableNameBuilder.toString();
|
||||
|
||||
if (INSERT_TYPE.equals(statementType)) {
|
||||
sql = generateInsert(jsonNode, attributes, fqTableName, schema, translateFieldNames, ignoreUnmappedFields,
|
||||
failUnmappedColumns, warningUnmappedColumns, escapeColumnNames, quoteTableName, attributePrefix);
|
||||
} else if (UPDATE_TYPE.equals(statementType)) {
|
||||
sql = generateUpdate(jsonNode, attributes, fqTableName, updateKeys, schema, translateFieldNames, ignoreUnmappedFields,
|
||||
failUnmappedColumns, warningUnmappedColumns, escapeColumnNames, quoteTableName, attributePrefix);
|
||||
} else {
|
||||
sql = generateDelete(jsonNode, attributes, fqTableName, schema, translateFieldNames, ignoreUnmappedFields,
|
||||
failUnmappedColumns, warningUnmappedColumns, escapeColumnNames, quoteTableName, attributePrefix);
|
||||
}
|
||||
} catch (final ProcessException pe) {
|
||||
getLogger().error("Failed to convert {} to a SQL {} statement due to {}; routing to failure",
|
||||
flowFile, statementType, pe.toString(), pe);
|
||||
session.remove(created);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
return;
|
||||
}
|
||||
|
||||
FlowFile sqlFlowFile = session.create(flowFile);
|
||||
created.add(sqlFlowFile);
|
||||
|
||||
sqlFlowFile = session.write(sqlFlowFile, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
out.write(sql.getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
});
|
||||
|
||||
attributes.put(CoreAttributes.MIME_TYPE.key(), "text/plain");
|
||||
attributes.put(attributePrefix + ".table", tableName);
|
||||
attributes.put(FRAGMENT_ID.key(), fragmentIdentifier);
|
||||
attributes.put(FRAGMENT_COUNT.key(), String.valueOf(arrayNode.size()));
|
||||
attributes.put(FRAGMENT_INDEX.key(), String.valueOf(i));
|
||||
|
||||
if (catalog != null) {
|
||||
attributes.put(attributePrefix + ".catalog", catalog);
|
||||
}
|
||||
|
||||
sqlFlowFile = session.putAllAttributes(sqlFlowFile, attributes);
|
||||
session.transfer(sqlFlowFile, REL_SQL);
|
||||
}
|
||||
|
||||
FlowFile newFlowFile = copyAttributesToOriginal(session, flowFile, fragmentIdentifier, arrayNode.size());
|
||||
session.transfer(newFlowFile, REL_ORIGINAL);
|
||||
}
|
||||
|
||||
private Set<String> getNormalizedColumnNames(final JsonNode node, final boolean translateFieldNames) {
|
||||
final Set<String> normalizedFieldNames = new HashSet<>();
|
||||
final Iterator<String> fieldNameItr = node.fieldNames();
|
||||
while (fieldNameItr.hasNext()) {
|
||||
normalizedFieldNames.add(normalizeColumnName(fieldNameItr.next(), translateFieldNames));
|
||||
}
|
||||
|
||||
return normalizedFieldNames;
|
||||
}
|
||||
|
||||
private String generateInsert(final JsonNode rootNode, final Map<String, String> attributes, final String tableName,
|
||||
final TableSchema schema, final boolean translateFieldNames, final boolean ignoreUnmappedFields, final boolean failUnmappedColumns,
|
||||
final boolean warningUnmappedColumns, boolean escapeColumnNames, boolean quoteTableName, final String attributePrefix) {
|
||||
|
||||
final Set<String> normalizedFieldNames = getNormalizedColumnNames(rootNode, translateFieldNames);
|
||||
for (final String requiredColName : schema.getRequiredColumnNames()) {
|
||||
final String normalizedColName = normalizeColumnName(requiredColName, translateFieldNames);
|
||||
if (!normalizedFieldNames.contains(normalizedColName)) {
|
||||
String missingColMessage = "JSON does not have a value for the Required column '" + requiredColName + "'";
|
||||
if (failUnmappedColumns) {
|
||||
getLogger().error(missingColMessage);
|
||||
throw new ProcessException(missingColMessage);
|
||||
} else if (warningUnmappedColumns) {
|
||||
getLogger().warn(missingColMessage);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final StringBuilder sqlBuilder = new StringBuilder();
|
||||
int fieldCount = 0;
|
||||
sqlBuilder.append("INSERT INTO ");
|
||||
if (quoteTableName) {
|
||||
sqlBuilder.append(schema.getQuotedIdentifierString())
|
||||
.append(tableName)
|
||||
.append(schema.getQuotedIdentifierString());
|
||||
} else {
|
||||
sqlBuilder.append(tableName);
|
||||
}
|
||||
sqlBuilder.append(" (");
|
||||
|
||||
// iterate over all of the elements in the JSON, building the SQL statement by adding the column names, as well as
|
||||
// adding the column value to a "<sql>.args.N.value" attribute and the type of a "<sql>.args.N.type" attribute add the
|
||||
// columns that we are inserting into
|
||||
final Iterator<String> fieldNames = rootNode.fieldNames();
|
||||
while (fieldNames.hasNext()) {
|
||||
final String fieldName = fieldNames.next();
|
||||
|
||||
final ColumnDescription desc = schema.getColumns().get(normalizeColumnName(fieldName, translateFieldNames));
|
||||
if (desc == null && !ignoreUnmappedFields) {
|
||||
throw new ProcessException("Cannot map JSON field '" + fieldName + "' to any column in the database");
|
||||
}
|
||||
|
||||
if (desc != null) {
|
||||
if (fieldCount++ > 0) {
|
||||
sqlBuilder.append(", ");
|
||||
}
|
||||
|
||||
if (escapeColumnNames) {
|
||||
sqlBuilder.append(schema.getQuotedIdentifierString())
|
||||
.append(desc.getColumnName())
|
||||
.append(schema.getQuotedIdentifierString());
|
||||
} else {
|
||||
sqlBuilder.append(desc.getColumnName());
|
||||
}
|
||||
|
||||
final int sqlType = desc.getDataType();
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".type", String.valueOf(sqlType));
|
||||
|
||||
final Integer colSize = desc.getColumnSize();
|
||||
final JsonNode fieldNode = rootNode.get(fieldName);
|
||||
if (!fieldNode.isNull()) {
|
||||
String fieldValue = createSqlStringValue(fieldNode, colSize, sqlType);
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".value", fieldValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// complete the SQL statements by adding ?'s for all of the values to be escaped.
|
||||
sqlBuilder.append(") VALUES (");
|
||||
for (int i = 0; i < fieldCount; i++) {
|
||||
if (i > 0) {
|
||||
sqlBuilder.append(", ");
|
||||
}
|
||||
|
||||
sqlBuilder.append("?");
|
||||
}
|
||||
sqlBuilder.append(")");
|
||||
|
||||
if (fieldCount == 0) {
|
||||
throw new ProcessException("None of the fields in the JSON map to the columns defined by the " + tableName + " table");
|
||||
}
|
||||
|
||||
return sqlBuilder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Try to create correct SQL String representation of value.
|
||||
*
|
||||
*/
|
||||
protected static String createSqlStringValue(final JsonNode fieldNode, final Integer colSize, final int sqlType) {
|
||||
String fieldValue = fieldNode.asText();
|
||||
|
||||
switch (sqlType) {
|
||||
|
||||
// only "true" is considered true, everything else is false
|
||||
case Types.BOOLEAN:
|
||||
fieldValue = Boolean.valueOf(fieldValue).toString();
|
||||
break;
|
||||
|
||||
// Don't truncate numeric types.
|
||||
case Types.BIT:
|
||||
case Types.TINYINT:
|
||||
case Types.SMALLINT:
|
||||
case Types.INTEGER:
|
||||
case Types.BIGINT:
|
||||
case Types.REAL:
|
||||
case Types.FLOAT:
|
||||
case Types.DOUBLE:
|
||||
case Types.DECIMAL:
|
||||
case Types.NUMERIC:
|
||||
if (fieldNode.isBoolean()) {
|
||||
// Convert boolean to number representation for databases those don't support boolean type.
|
||||
fieldValue = fieldNode.asBoolean() ? "1" : "0";
|
||||
}
|
||||
break;
|
||||
|
||||
// Don't truncate DATE, TIME and TIMESTAMP types. We assume date and time is already correct in long representation.
|
||||
// Specifically, milliseconds since January 1, 1970, 00:00:00 GMT
|
||||
// However, for TIMESTAMP, PutSQL accepts optional timestamp format via FlowFile attribute.
|
||||
// See PutSQL.setParameter method and NIFI-3430 for detail.
|
||||
// Alternatively, user can use JSONTreeReader and PutDatabaseRecord to handle date format more efficiently.
|
||||
case Types.DATE:
|
||||
case Types.TIME:
|
||||
case Types.TIMESTAMP:
|
||||
break;
|
||||
|
||||
// Truncate string data types only.
|
||||
case Types.CHAR:
|
||||
case Types.VARCHAR:
|
||||
case Types.LONGVARCHAR:
|
||||
case Types.NCHAR:
|
||||
case Types.NVARCHAR:
|
||||
case Types.LONGNVARCHAR:
|
||||
if (colSize != null && fieldValue.length() > colSize) {
|
||||
fieldValue = fieldValue.substring(0, colSize);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
return fieldValue;
|
||||
}
|
||||
|
||||
private String generateUpdate(final JsonNode rootNode, final Map<String, String> attributes, final String tableName, final String updateKeys,
|
||||
final TableSchema schema, final boolean translateFieldNames, final boolean ignoreUnmappedFields, final boolean failUnmappedColumns,
|
||||
final boolean warningUnmappedColumns, boolean escapeColumnNames, boolean quoteTableName, final String attributePrefix) {
|
||||
|
||||
final Set<String> updateKeyNames;
|
||||
if (updateKeys == null) {
|
||||
updateKeyNames = schema.getPrimaryKeyColumnNames();
|
||||
} else {
|
||||
updateKeyNames = new HashSet<>();
|
||||
for (final String updateKey : updateKeys.split(",")) {
|
||||
updateKeyNames.add(updateKey.trim());
|
||||
}
|
||||
}
|
||||
|
||||
if (updateKeyNames.isEmpty()) {
|
||||
throw new ProcessException("Table '" + tableName + "' does not have a Primary Key and no Update Keys were specified");
|
||||
}
|
||||
|
||||
final StringBuilder sqlBuilder = new StringBuilder();
|
||||
int fieldCount = 0;
|
||||
sqlBuilder.append("UPDATE ");
|
||||
if (quoteTableName) {
|
||||
sqlBuilder.append(schema.getQuotedIdentifierString())
|
||||
.append(tableName)
|
||||
.append(schema.getQuotedIdentifierString());
|
||||
} else {
|
||||
sqlBuilder.append(tableName);
|
||||
}
|
||||
|
||||
sqlBuilder.append(" SET ");
|
||||
|
||||
|
||||
// Create a Set of all normalized Update Key names, and ensure that there is a field in the JSON
|
||||
// for each of the Update Key fields.
|
||||
final Set<String> normalizedFieldNames = getNormalizedColumnNames(rootNode, translateFieldNames);
|
||||
final Set<String> normalizedUpdateNames = new HashSet<>();
|
||||
for (final String uk : updateKeyNames) {
|
||||
final String normalizedUK = normalizeColumnName(uk, translateFieldNames);
|
||||
normalizedUpdateNames.add(normalizedUK);
|
||||
|
||||
if (!normalizedFieldNames.contains(normalizedUK)) {
|
||||
String missingColMessage = "JSON does not have a value for the " + (updateKeys == null ? "Primary" : "Update") + "Key column '" + uk + "'";
|
||||
if (failUnmappedColumns) {
|
||||
getLogger().error(missingColMessage);
|
||||
throw new ProcessException(missingColMessage);
|
||||
} else if (warningUnmappedColumns) {
|
||||
getLogger().warn(missingColMessage);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// iterate over all of the elements in the JSON, building the SQL statement by adding the column names, as well as
|
||||
// adding the column value to a "<sql>.args.N.value" attribute and the type of a "<sql>.args.N.type" attribute add the
|
||||
// columns that we are inserting into
|
||||
Iterator<String> fieldNames = rootNode.fieldNames();
|
||||
while (fieldNames.hasNext()) {
|
||||
final String fieldName = fieldNames.next();
|
||||
|
||||
final String normalizedColName = normalizeColumnName(fieldName, translateFieldNames);
|
||||
final ColumnDescription desc = schema.getColumns().get(normalizedColName);
|
||||
if (desc == null) {
|
||||
if (!ignoreUnmappedFields) {
|
||||
throw new ProcessException("Cannot map JSON field '" + fieldName + "' to any column in the database");
|
||||
} else {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
// Check if this column is an Update Key. If so, skip it for now. We will come
|
||||
// back to it after we finish the SET clause
|
||||
if (normalizedUpdateNames.contains(normalizedColName)) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (fieldCount++ > 0) {
|
||||
sqlBuilder.append(", ");
|
||||
}
|
||||
|
||||
if (escapeColumnNames) {
|
||||
sqlBuilder.append(schema.getQuotedIdentifierString())
|
||||
.append(desc.getColumnName())
|
||||
.append(schema.getQuotedIdentifierString());
|
||||
} else {
|
||||
sqlBuilder.append(desc.getColumnName());
|
||||
}
|
||||
|
||||
sqlBuilder.append(" = ?");
|
||||
final int sqlType = desc.getDataType();
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".type", String.valueOf(sqlType));
|
||||
|
||||
final Integer colSize = desc.getColumnSize();
|
||||
|
||||
final JsonNode fieldNode = rootNode.get(fieldName);
|
||||
if (!fieldNode.isNull()) {
|
||||
String fieldValue = createSqlStringValue(fieldNode, colSize, sqlType);
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".value", fieldValue);
|
||||
}
|
||||
}
|
||||
|
||||
// Set the WHERE clause based on the Update Key values
|
||||
sqlBuilder.append(" WHERE ");
|
||||
|
||||
fieldNames = rootNode.fieldNames();
|
||||
int whereFieldCount = 0;
|
||||
while (fieldNames.hasNext()) {
|
||||
final String fieldName = fieldNames.next();
|
||||
|
||||
final String normalizedColName = normalizeColumnName(fieldName, translateFieldNames);
|
||||
final ColumnDescription desc = schema.getColumns().get(normalizedColName);
|
||||
if (desc == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Check if this column is a Update Key. If so, skip it for now. We will come
|
||||
// back to it after we finish the SET clause
|
||||
if (!normalizedUpdateNames.contains(normalizedColName)) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (whereFieldCount++ > 0) {
|
||||
sqlBuilder.append(" AND ");
|
||||
}
|
||||
fieldCount++;
|
||||
|
||||
if (escapeColumnNames) {
|
||||
sqlBuilder.append(schema.getQuotedIdentifierString())
|
||||
.append(normalizedColName)
|
||||
.append(schema.getQuotedIdentifierString());
|
||||
} else {
|
||||
sqlBuilder.append(normalizedColName);
|
||||
}
|
||||
sqlBuilder.append(" = ?");
|
||||
final int sqlType = desc.getDataType();
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".type", String.valueOf(sqlType));
|
||||
|
||||
final Integer colSize = desc.getColumnSize();
|
||||
String fieldValue = rootNode.get(fieldName).asText();
|
||||
if (colSize != null && fieldValue.length() > colSize) {
|
||||
fieldValue = fieldValue.substring(0, colSize);
|
||||
}
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".value", fieldValue);
|
||||
}
|
||||
|
||||
return sqlBuilder.toString();
|
||||
}
|
||||
|
||||
private String generateDelete(final JsonNode rootNode, final Map<String, String> attributes, final String tableName,
|
||||
final TableSchema schema, final boolean translateFieldNames, final boolean ignoreUnmappedFields, final boolean failUnmappedColumns,
|
||||
final boolean warningUnmappedColumns, boolean escapeColumnNames, boolean quoteTableName, final String attributePrefix) {
|
||||
final Set<String> normalizedFieldNames = getNormalizedColumnNames(rootNode, translateFieldNames);
|
||||
for (final String requiredColName : schema.getRequiredColumnNames()) {
|
||||
final String normalizedColName = normalizeColumnName(requiredColName, translateFieldNames);
|
||||
if (!normalizedFieldNames.contains(normalizedColName)) {
|
||||
String missingColMessage = "JSON does not have a value for the Required column '" + requiredColName + "'";
|
||||
if (failUnmappedColumns) {
|
||||
getLogger().error(missingColMessage);
|
||||
throw new ProcessException(missingColMessage);
|
||||
} else if (warningUnmappedColumns) {
|
||||
getLogger().warn(missingColMessage);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final StringBuilder sqlBuilder = new StringBuilder();
|
||||
int fieldCount = 0;
|
||||
sqlBuilder.append("DELETE FROM ");
|
||||
if (quoteTableName) {
|
||||
sqlBuilder.append(schema.getQuotedIdentifierString())
|
||||
.append(tableName)
|
||||
.append(schema.getQuotedIdentifierString());
|
||||
} else {
|
||||
sqlBuilder.append(tableName);
|
||||
}
|
||||
|
||||
sqlBuilder.append(" WHERE ");
|
||||
|
||||
// iterate over all of the elements in the JSON, building the SQL statement by adding the column names, as well as
|
||||
// adding the column value to a "<sql>.args.N.value" attribute and the type of a "<sql>.args.N.type" attribute add the
|
||||
// columns that we are inserting into
|
||||
final Iterator<String> fieldNames = rootNode.fieldNames();
|
||||
while (fieldNames.hasNext()) {
|
||||
final String fieldName = fieldNames.next();
|
||||
|
||||
final ColumnDescription desc = schema.getColumns().get(normalizeColumnName(fieldName, translateFieldNames));
|
||||
if (desc == null && !ignoreUnmappedFields) {
|
||||
throw new ProcessException("Cannot map JSON field '" + fieldName + "' to any column in the database");
|
||||
}
|
||||
|
||||
if (desc != null) {
|
||||
if (fieldCount++ > 0) {
|
||||
sqlBuilder.append(" AND ");
|
||||
}
|
||||
|
||||
if (escapeColumnNames) {
|
||||
sqlBuilder.append(schema.getQuotedIdentifierString())
|
||||
.append(desc.getColumnName())
|
||||
.append(schema.getQuotedIdentifierString());
|
||||
} else {
|
||||
sqlBuilder.append(desc.getColumnName());
|
||||
}
|
||||
sqlBuilder.append(" = ?");
|
||||
|
||||
final int sqlType = desc.getDataType();
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".type", String.valueOf(sqlType));
|
||||
|
||||
final Integer colSize = desc.getColumnSize();
|
||||
final JsonNode fieldNode = rootNode.get(fieldName);
|
||||
if (!fieldNode.isNull()) {
|
||||
String fieldValue = fieldNode.asText();
|
||||
if (colSize != null && fieldValue.length() > colSize) {
|
||||
fieldValue = fieldValue.substring(0, colSize);
|
||||
}
|
||||
attributes.put(attributePrefix + ".args." + fieldCount + ".value", fieldValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (fieldCount == 0) {
|
||||
throw new ProcessException("None of the fields in the JSON map to the columns defined by the " + tableName + " table");
|
||||
}
|
||||
|
||||
return sqlBuilder.toString();
|
||||
}
|
||||
|
||||
private static String normalizeColumnName(final String colName, final boolean translateColumnNames) {
|
||||
return translateColumnNames ? colName.toUpperCase().replace("_", "") : colName;
|
||||
}
|
||||
|
||||
private static class TableSchema {
|
||||
private List<String> requiredColumnNames;
|
||||
private Set<String> primaryKeyColumnNames;
|
||||
private Map<String, ColumnDescription> columns;
|
||||
private String quotedIdentifierString;
|
||||
|
||||
private TableSchema(final List<ColumnDescription> columnDescriptions, final boolean translateColumnNames,
|
||||
final Set<String> primaryKeyColumnNames, final String quotedIdentifierString) {
|
||||
this.columns = new HashMap<>();
|
||||
this.primaryKeyColumnNames = primaryKeyColumnNames;
|
||||
this.quotedIdentifierString = quotedIdentifierString;
|
||||
|
||||
this.requiredColumnNames = new ArrayList<>();
|
||||
for (final ColumnDescription desc : columnDescriptions) {
|
||||
columns.put(ConvertJSONToSQL.normalizeColumnName(desc.columnName, translateColumnNames), desc);
|
||||
if (desc.isRequired()) {
|
||||
requiredColumnNames.add(desc.columnName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, ColumnDescription> getColumns() {
|
||||
return columns;
|
||||
}
|
||||
|
||||
public List<String> getRequiredColumnNames() {
|
||||
return requiredColumnNames;
|
||||
}
|
||||
|
||||
public Set<String> getPrimaryKeyColumnNames() {
|
||||
return primaryKeyColumnNames;
|
||||
}
|
||||
|
||||
public String getQuotedIdentifierString() {
|
||||
return quotedIdentifierString;
|
||||
}
|
||||
|
||||
public static TableSchema from(final Connection conn, final String catalog, final String schema, final String tableName,
|
||||
final boolean translateColumnNames, final boolean includePrimaryKeys) throws SQLException {
|
||||
final DatabaseMetaData dmd = conn.getMetaData();
|
||||
|
||||
try (final ResultSet colrs = dmd.getColumns(catalog, schema, tableName, "%")) {
|
||||
final List<ColumnDescription> cols = new ArrayList<>();
|
||||
while (colrs.next()) {
|
||||
final ColumnDescription col = ColumnDescription.from(colrs);
|
||||
cols.add(col);
|
||||
}
|
||||
|
||||
final Set<String> primaryKeyColumns = new HashSet<>();
|
||||
if (includePrimaryKeys) {
|
||||
try (final ResultSet pkrs = conn.getMetaData().getPrimaryKeys(catalog, null, tableName)) {
|
||||
|
||||
while (pkrs.next()) {
|
||||
final String colName = pkrs.getString("COLUMN_NAME");
|
||||
primaryKeyColumns.add(normalizeColumnName(colName, translateColumnNames));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return new TableSchema(cols, translateColumnNames, primaryKeyColumns, dmd.getIdentifierQuoteString());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class ColumnDescription {
|
||||
private final String columnName;
|
||||
private final int dataType;
|
||||
private final boolean required;
|
||||
private final Integer columnSize;
|
||||
|
||||
private ColumnDescription(final String columnName, final int dataType, final boolean required, final Integer columnSize) {
|
||||
this.columnName = columnName;
|
||||
this.dataType = dataType;
|
||||
this.required = required;
|
||||
this.columnSize = columnSize;
|
||||
}
|
||||
|
||||
public int getDataType() {
|
||||
return dataType;
|
||||
}
|
||||
|
||||
public Integer getColumnSize() {
|
||||
return columnSize;
|
||||
}
|
||||
|
||||
public String getColumnName() {
|
||||
return columnName;
|
||||
}
|
||||
|
||||
public boolean isRequired() {
|
||||
return required;
|
||||
}
|
||||
|
||||
public static ColumnDescription from(final ResultSet resultSet) throws SQLException {
|
||||
final ResultSetMetaData md = resultSet.getMetaData();
|
||||
List<String> columns = new ArrayList<>();
|
||||
|
||||
for (int i = 1; i < md.getColumnCount() + 1; i++) {
|
||||
columns.add(md.getColumnName(i));
|
||||
}
|
||||
// COLUMN_DEF must be read first to work around Oracle bug, see NIFI-4279 for details
|
||||
final String defaultValue = resultSet.getString("COLUMN_DEF");
|
||||
final String columnName = resultSet.getString("COLUMN_NAME");
|
||||
final int dataType = resultSet.getInt("DATA_TYPE");
|
||||
final int colSize = resultSet.getInt("COLUMN_SIZE");
|
||||
|
||||
final String nullableValue = resultSet.getString("IS_NULLABLE");
|
||||
final boolean isNullable = "YES".equalsIgnoreCase(nullableValue) || nullableValue.isEmpty();
|
||||
|
||||
String autoIncrementValue = "NO";
|
||||
|
||||
if (columns.contains("IS_AUTOINCREMENT")) {
|
||||
autoIncrementValue = resultSet.getString("IS_AUTOINCREMENT");
|
||||
}
|
||||
|
||||
final boolean isAutoIncrement = "YES".equalsIgnoreCase(autoIncrementValue);
|
||||
final boolean required = !isNullable && !isAutoIncrement && defaultValue == null;
|
||||
|
||||
return new ColumnDescription(columnName, dataType, required, colSize == 0 ? null : colSize);
|
||||
}
|
||||
}
|
||||
|
||||
private static class SchemaKey {
|
||||
private final String catalog;
|
||||
private final String tableName;
|
||||
|
||||
public SchemaKey(final String catalog, final String tableName) {
|
||||
this.catalog = catalog;
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + ((catalog == null) ? 0 : catalog.hashCode());
|
||||
result = prime * result + ((tableName == null) ? 0 : tableName.hashCode());
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final SchemaKey other = (SchemaKey) obj;
|
||||
if (catalog == null) {
|
||||
if (other.catalog != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!catalog.equals(other.catalog)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
if (tableName == null) {
|
||||
if (other.tableName != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!tableName.equals(other.tableName)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,322 +0,0 @@
|
|||
/*
|
||||
* 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.standard;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
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.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.event.transport.EventException;
|
||||
import org.apache.nifi.event.transport.EventServer;
|
||||
import org.apache.nifi.event.transport.netty.NettyEventServerFactory;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.flowfile.attributes.FlowFileAttributeKey;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.listen.EventBatcher;
|
||||
import org.apache.nifi.processor.util.listen.FlowFileEventBatch;
|
||||
import org.apache.nifi.processor.util.listen.ListenerProperties;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMessage;
|
||||
import org.apache.nifi.processors.standard.relp.handler.RELPMessageServerFactory;
|
||||
import org.apache.nifi.remote.io.socket.NetworkUtils;
|
||||
import org.apache.nifi.security.util.ClientAuth;
|
||||
import org.apache.nifi.ssl.RestrictedSSLContextService;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"listen", "relp", "tcp", "logs"})
|
||||
@CapabilityDescription("Listens for RELP 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 RELP frames. In the case where the RELP frames contain syslog messages, the " +
|
||||
"output of this processor can be sent to a ParseSyslog processor for further processing.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "relp.command", description = "The command of the RELP frames."),
|
||||
@WritesAttribute(attribute = "relp.sender", description = "The sending host of the messages."),
|
||||
@WritesAttribute(attribute = "relp.port", description = "The sending port the messages were received over."),
|
||||
@WritesAttribute(attribute = "relp.txnr", description = "The transaction number of the message. Only included if <Batch Size> is 1."),
|
||||
@WritesAttribute(attribute = "mime.type", description = "The mime.type of the content which is text/plain")
|
||||
})
|
||||
@SeeAlso({ParseSyslog.class})
|
||||
public class ListenRELP extends AbstractProcessor {
|
||||
|
||||
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("SSL Context Service")
|
||||
.displayName("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.")
|
||||
.required(false)
|
||||
.identifiesControllerService(RestrictedSSLContextService.class)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder()
|
||||
.name("Client Auth")
|
||||
.displayName("Client Auth")
|
||||
.description("The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.")
|
||||
.required(false)
|
||||
.allowableValues(ClientAuth.values())
|
||||
.defaultValue(ClientAuth.REQUIRED.name())
|
||||
.build();
|
||||
|
||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("Messages received successfully will be sent out this relationship.")
|
||||
.build();
|
||||
|
||||
protected List<PropertyDescriptor> descriptors;
|
||||
protected Set<Relationship> relationships;
|
||||
protected volatile int port;
|
||||
protected volatile BlockingQueue<RELPMessage> events;
|
||||
protected volatile BlockingQueue<RELPMessage> errorEvents;
|
||||
protected volatile EventServer eventServer;
|
||||
protected volatile byte[] messageDemarcatorBytes;
|
||||
protected volatile EventBatcher eventBatcher;
|
||||
|
||||
@OnScheduled
|
||||
public void onScheduled(ProcessContext context) throws IOException {
|
||||
int workerThreads = context.getProperty(ListenerProperties.WORKER_THREADS).asInteger();
|
||||
int bufferSize = context.getProperty(ListenerProperties.RECV_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
final String networkInterface = context.getProperty(ListenerProperties.NETWORK_INTF_NAME).evaluateAttributeExpressions().getValue();
|
||||
InetAddress hostname = NetworkUtils.getInterfaceAddress(networkInterface);
|
||||
Charset charset = Charset.forName(context.getProperty(ListenerProperties.CHARSET).getValue());
|
||||
port = context.getProperty(ListenerProperties.PORT).evaluateAttributeExpressions().asInteger();
|
||||
events = new LinkedBlockingQueue<>(context.getProperty(ListenerProperties.MAX_MESSAGE_QUEUE_SIZE).asInteger());
|
||||
errorEvents = new LinkedBlockingQueue<>();
|
||||
eventBatcher = getEventBatcher();
|
||||
|
||||
final String msgDemarcator = getMessageDemarcator(context);
|
||||
messageDemarcatorBytes = msgDemarcator.getBytes(charset);
|
||||
final NettyEventServerFactory eventFactory = getNettyEventServerFactory(hostname, port, charset, events);
|
||||
eventFactory.setSocketReceiveBuffer(bufferSize);
|
||||
eventFactory.setWorkerThreads(workerThreads);
|
||||
configureFactoryForSsl(context, eventFactory);
|
||||
|
||||
try {
|
||||
eventServer = eventFactory.getEventServer();
|
||||
} catch (EventException e) {
|
||||
getLogger().error("Failed to bind to [{}:{}].", hostname.getHostAddress(), port);
|
||||
}
|
||||
}
|
||||
|
||||
public int getListeningPort() {
|
||||
return eventServer == null ? 0 : eventServer.getListeningPort();
|
||||
}
|
||||
|
||||
@OnStopped
|
||||
public void stopped() {
|
||||
if (eventServer != null) {
|
||||
eventServer.shutdown();
|
||||
eventServer = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void init(final ProcessorInitializationContext context) {
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
descriptors.add(ListenerProperties.NETWORK_INTF_NAME);
|
||||
descriptors.add(ListenerProperties.PORT);
|
||||
descriptors.add(ListenerProperties.RECV_BUFFER_SIZE);
|
||||
descriptors.add(ListenerProperties.MAX_MESSAGE_QUEUE_SIZE);
|
||||
descriptors.add(ListenerProperties.MAX_SOCKET_BUFFER_SIZE);
|
||||
descriptors.add(ListenerProperties.CHARSET);
|
||||
descriptors.add(ListenerProperties.WORKER_THREADS);
|
||||
descriptors.add(ListenerProperties.MAX_BATCH_SIZE);
|
||||
descriptors.add(ListenerProperties.MESSAGE_DELIMITER);
|
||||
descriptors.add(SSL_CONTEXT_SERVICE);
|
||||
descriptors.add(CLIENT_AUTH);
|
||||
this.descriptors = Collections.unmodifiableList(descriptors);
|
||||
|
||||
final Set<Relationship> relationships = new HashSet<>();
|
||||
relationships.add(REL_SUCCESS);
|
||||
this.relationships = Collections.unmodifiableSet(relationships);
|
||||
}
|
||||
|
||||
@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);
|
||||
final String clientAuth = validationContext.getProperty(CLIENT_AUTH).getValue();
|
||||
if (sslContextService != null && StringUtils.isBlank(clientAuth)) {
|
||||
results.add(new ValidationResult.Builder()
|
||||
.explanation("Client Auth must be provided when using TLS/SSL")
|
||||
.valid(false).subject("Client Auth").build());
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Set<Relationship> getRelationships() {
|
||||
return this.relationships;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
private void configureFactoryForSsl(final ProcessContext context, final NettyEventServerFactory eventFactory) {
|
||||
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
||||
if (sslContextService != null) {
|
||||
final String clientAuthValue = context.getProperty(CLIENT_AUTH).getValue();
|
||||
SSLContext sslContext = sslContextService.createContext();
|
||||
if (sslContext != null) {
|
||||
eventFactory.setSslContext(sslContext);
|
||||
eventFactory.setClientAuth(ClientAuth.valueOf(clientAuthValue));
|
||||
}
|
||||
} else {
|
||||
eventFactory.setSslContext(null);
|
||||
}
|
||||
}
|
||||
|
||||
protected Map<String, String> getAttributes(FlowFileEventBatch batch) {
|
||||
final List<RELPMessage> 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 String command = events.get(0).getCommand();
|
||||
|
||||
final int numAttributes = events.size() == 1 ? 5 : 4;
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>(numAttributes);
|
||||
attributes.put(RELPAttributes.COMMAND.key(), command);
|
||||
attributes.put(RELPAttributes.SENDER.key(), sender);
|
||||
attributes.put(RELPAttributes.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(RELPAttributes.TXNR.key(), String.valueOf(events.get(0).getTxnr()));
|
||||
}
|
||||
return attributes;
|
||||
}
|
||||
|
||||
protected String getTransitUri(FlowFileEventBatch batch) {
|
||||
final List<RELPMessage> events = batch.getEvents();
|
||||
final String sender = events.get(0).getSender();
|
||||
final String senderHost = sender.startsWith("/") && sender.length() > 1 ? sender.substring(1) : sender;
|
||||
final String transitUri = new StringBuilder().append("relp").append("://").append(senderHost).append(":")
|
||||
.append(port).toString();
|
||||
return transitUri;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
|
||||
EventBatcher eventBatcher = getEventBatcher();
|
||||
|
||||
final int batchSize = context.getProperty(ListenerProperties.MAX_BATCH_SIZE).asInteger();
|
||||
Map<String, FlowFileEventBatch> batches = eventBatcher.getBatches(session, batchSize, messageDemarcatorBytes);
|
||||
processEvents(session, batches);
|
||||
}
|
||||
|
||||
private void processEvents(final ProcessSession session, final Map<String, FlowFileEventBatch> batches) {
|
||||
for (Map.Entry<String, FlowFileEventBatch> entry : batches.entrySet()) {
|
||||
FlowFile flowFile = entry.getValue().getFlowFile();
|
||||
final List<RELPMessage> events = entry.getValue().getEvents();
|
||||
|
||||
if (flowFile.getSize() == 0L || events.size() == 0) {
|
||||
session.remove(flowFile);
|
||||
getLogger().debug("No data written to FlowFile from batch {}; removing FlowFile", entry.getKey());
|
||||
continue;
|
||||
}
|
||||
|
||||
final Map<String, String> attributes = getAttributes(entry.getValue());
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
getLogger().debug("Transferring {} to success", flowFile);
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
session.adjustCounter("FlowFiles Transferred to Success", 1L, false);
|
||||
|
||||
// the sender and command will be the same for all events based on the batch key
|
||||
final String transitUri = getTransitUri(entry.getValue());
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri);
|
||||
|
||||
}
|
||||
session.commitAsync();
|
||||
}
|
||||
|
||||
private String getRELPBatchKey(final RELPMessage event) {
|
||||
return event.getSender() + "_" + event.getCommand();
|
||||
}
|
||||
|
||||
private EventBatcher getEventBatcher() {
|
||||
return new EventBatcher<RELPMessage>(getLogger(), events, errorEvents) {
|
||||
@Override
|
||||
protected String getBatchKey(RELPMessage event) {
|
||||
return getRELPBatchKey(event);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public enum RELPAttributes implements FlowFileAttributeKey {
|
||||
TXNR("relp.txnr"),
|
||||
COMMAND("relp.command"),
|
||||
SENDER("relp.sender"),
|
||||
PORT("relp.port");
|
||||
|
||||
private final String key;
|
||||
|
||||
RELPAttributes(String key) {
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String key() {
|
||||
return key;
|
||||
}
|
||||
}
|
||||
|
||||
private NettyEventServerFactory getNettyEventServerFactory(final InetAddress hostname, final int port, final Charset charset, final BlockingQueue events) {
|
||||
return new RELPMessageServerFactory(getLogger(), hostname, port, charset, events);
|
||||
}
|
||||
|
||||
private String getMessageDemarcator(final ProcessContext context) {
|
||||
return context.getProperty(ListenerProperties.MESSAGE_DELIMITER)
|
||||
.getValue()
|
||||
.replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
||||
}
|
||||
}
|
|
@ -1,501 +0,0 @@
|
|||
/*
|
||||
* 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.standard;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.InetAddress;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.NetworkInterface;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.channels.ServerSocketChannel;
|
||||
import java.security.cert.Certificate;
|
||||
import java.security.cert.X509Certificate;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLPeerUnverifiedException;
|
||||
import javax.net.ssl.SSLSession;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
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.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.expression.ExpressionLanguageScope;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processor.util.listen.ListenerProperties;
|
||||
import org.apache.nifi.record.listen.SSLSocketChannelRecordReader;
|
||||
import org.apache.nifi.record.listen.SocketChannelRecordReader;
|
||||
import org.apache.nifi.record.listen.SocketChannelRecordReaderDispatcher;
|
||||
import org.apache.nifi.security.cert.PrincipalFormatter;
|
||||
import org.apache.nifi.security.cert.StandardPrincipalFormatter;
|
||||
import org.apache.nifi.security.util.ClientAuth;
|
||||
import org.apache.nifi.serialization.RecordReader;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
import org.apache.nifi.serialization.RecordSetWriter;
|
||||
import org.apache.nifi.serialization.RecordSetWriterFactory;
|
||||
import org.apache.nifi.serialization.WriteResult;
|
||||
import org.apache.nifi.serialization.record.Record;
|
||||
import org.apache.nifi.serialization.record.RecordSchema;
|
||||
import org.apache.nifi.ssl.RestrictedSSLContextService;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
|
||||
import static org.apache.nifi.processor.util.listen.ListenerProperties.NETWORK_INTF_NAME;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"listen", "tcp", "record", "tls", "ssl"})
|
||||
@CapabilityDescription("Listens for incoming TCP connections and reads data from each connection using a configured record " +
|
||||
"reader, and writes the records to a flow file using a configured record writer. The type of record reader selected will " +
|
||||
"determine how clients are expected to send data. For example, when using a Grok reader to read logs, a client can keep an " +
|
||||
"open connection and continuously stream data, but when using an JSON reader, the client cannot send an array of JSON " +
|
||||
"documents and then send another array on the same connection, as the reader would be in a bad state at that point. Records " +
|
||||
"will be read from the connection in blocking mode, and will timeout according to the Read Timeout specified in the processor. " +
|
||||
"If the read times out, or if any other error is encountered when reading, the connection will be closed, and any records " +
|
||||
"read up to that point will be handled according to the configured Read Error Strategy (Discard or Transfer). In cases where " +
|
||||
"clients are keeping a connection open, the concurrent tasks for the processor should be adjusted to match the Max Number of " +
|
||||
"TCP Connections allowed, so that there is a task processing each connection. " +
|
||||
"The processor can be configured to use an SSL Context Service to only allow secure connections. " +
|
||||
"When connected clients present certificates for mutual TLS authentication, the Distinguished Names of the client certificate's " +
|
||||
"issuer and subject are added to the outgoing FlowFiles as attributes. " +
|
||||
"The processor does not perform authorization based on Distinguished Name values, but since these values " +
|
||||
"are attached to the outgoing FlowFiles, authorization can be implemented based on these attributes.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "tcp.sender", description = "The host that sent the data."),
|
||||
@WritesAttribute(attribute = "tcp.port", description = "The port that the processor accepted the connection on."),
|
||||
@WritesAttribute(attribute = "record.count", description = "The number of records written to the flow file."),
|
||||
@WritesAttribute(attribute = "mime.type", description = "The mime-type of the writer used to write the records to the flow file."),
|
||||
@WritesAttribute(attribute = "client.certificate.issuer.dn", description = "For connections using mutual TLS, the Distinguished Name of the " +
|
||||
"Certificate Authority that issued the client's certificate " +
|
||||
"is attached to the FlowFile."),
|
||||
@WritesAttribute(attribute = "client.certificate.subject.dn", description = "For connections using mutual TLS, the Distinguished Name of the " +
|
||||
"client certificate's owner (subject) is attached to the FlowFile.")
|
||||
})
|
||||
public class ListenTCPRecord extends AbstractProcessor {
|
||||
private static final String CLIENT_CERTIFICATE_SUBJECT_DN_ATTRIBUTE = "client.certificate.subject.dn";
|
||||
private static final String CLIENT_CERTIFICATE_ISSUER_DN_ATTRIBUTE = "client.certificate.issuer.dn";
|
||||
|
||||
static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
|
||||
.name("port")
|
||||
.displayName("Port")
|
||||
.description("The port to listen on for communication.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.PORT_VALIDATOR)
|
||||
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor READ_TIMEOUT = new PropertyDescriptor.Builder()
|
||||
.name("read-timeout")
|
||||
.displayName("Read Timeout")
|
||||
.description("The amount of time to wait before timing out when reading from a connection.")
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
.defaultValue("10 seconds")
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor MAX_SOCKET_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("max-size-socket-buffer")
|
||||
.displayName("Max Size of Socket Buffer")
|
||||
.description("The maximum size of the socket buffer that should be used. This is a suggestion to the Operating System " +
|
||||
"to indicate how big the socket buffer should be. If this value is set too low, the buffer may fill up before " +
|
||||
"the data can be read, and incoming data will be dropped.")
|
||||
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||
.defaultValue("1 MB")
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor MAX_CONNECTIONS = new PropertyDescriptor.Builder()
|
||||
.name("max-number-tcp-connections")
|
||||
.displayName("Max Number of TCP Connections")
|
||||
.description("The maximum number of concurrent TCP connections to accept. In cases where clients are keeping a connection open, " +
|
||||
"the concurrent tasks for the processor should be adjusted to match the Max Number of TCP Connections allowed, so that there " +
|
||||
"is a task processing each connection.")
|
||||
.addValidator(StandardValidators.createLongValidator(1, 65535, true))
|
||||
.defaultValue("2")
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
|
||||
.name("record-reader")
|
||||
.displayName("Record Reader")
|
||||
.description("The Record Reader to use for incoming FlowFiles")
|
||||
.identifiesControllerService(RecordReaderFactory.class)
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
|
||||
.name("record-writer")
|
||||
.displayName("Record Writer")
|
||||
.description("The Record Writer to use in order to serialize the data before writing to a FlowFile")
|
||||
.identifiesControllerService(RecordSetWriterFactory.class)
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final AllowableValue ERROR_HANDLING_DISCARD = new AllowableValue("Discard", "Discard", "Discards any records already received and closes the connection.");
|
||||
static final AllowableValue ERROR_HANDLING_TRANSFER = new AllowableValue("Transfer", "Transfer", "Transfers any records already received and closes the connection.");
|
||||
|
||||
static final PropertyDescriptor READER_ERROR_HANDLING_STRATEGY = new PropertyDescriptor.Builder()
|
||||
.name("reader-error-handling-strategy")
|
||||
.displayName("Read Error Strategy")
|
||||
.description("Indicates how to deal with an error while reading the next record from a connection, when previous records have already been read from the connection.")
|
||||
.required(true)
|
||||
.allowableValues(ERROR_HANDLING_TRANSFER, ERROR_HANDLING_DISCARD)
|
||||
.defaultValue(ERROR_HANDLING_TRANSFER.getValue())
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor RECORD_BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("record-batch-size")
|
||||
.displayName("Record Batch Size")
|
||||
.description("The maximum number of records to write to a single FlowFile.")
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.defaultValue("1000")
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("ssl-context-service")
|
||||
.displayName("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.")
|
||||
.required(false)
|
||||
.identifiesControllerService(RestrictedSSLContextService.class)
|
||||
.build();
|
||||
|
||||
static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder()
|
||||
.name("client-auth")
|
||||
.displayName("Client Auth")
|
||||
.description("The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.")
|
||||
.required(false)
|
||||
.allowableValues(ClientAuth.values())
|
||||
.defaultValue(ClientAuth.REQUIRED.name())
|
||||
.build();
|
||||
|
||||
static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("Messages received successfully will be sent out this relationship.")
|
||||
.build();
|
||||
|
||||
static final List<PropertyDescriptor> PROPERTIES;
|
||||
static {
|
||||
final List<PropertyDescriptor> props = new ArrayList<>();
|
||||
props.add(ListenerProperties.NETWORK_INTF_NAME);
|
||||
props.add(PORT);
|
||||
props.add(MAX_SOCKET_BUFFER_SIZE);
|
||||
props.add(MAX_CONNECTIONS);
|
||||
props.add(READ_TIMEOUT);
|
||||
props.add(RECORD_READER);
|
||||
props.add(RECORD_WRITER);
|
||||
props.add(READER_ERROR_HANDLING_STRATEGY);
|
||||
props.add(RECORD_BATCH_SIZE);
|
||||
props.add(SSL_CONTEXT_SERVICE);
|
||||
props.add(CLIENT_AUTH);
|
||||
PROPERTIES = Collections.unmodifiableList(props);
|
||||
}
|
||||
|
||||
static final Set<Relationship> RELATIONSHIPS;
|
||||
static {
|
||||
final Set<Relationship> rels = new HashSet<>();
|
||||
rels.add(REL_SUCCESS);
|
||||
RELATIONSHIPS = Collections.unmodifiableSet(rels);
|
||||
}
|
||||
|
||||
static final int POLL_TIMEOUT_MS = 20;
|
||||
|
||||
private volatile int port;
|
||||
private volatile SocketChannelRecordReaderDispatcher dispatcher;
|
||||
private final BlockingQueue<SocketChannelRecordReader> socketReaders = new LinkedBlockingQueue<>();
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return RELATIONSHIPS;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return PROPERTIES;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
|
||||
final List<ValidationResult> results = new ArrayList<>();
|
||||
|
||||
final String clientAuth = validationContext.getProperty(CLIENT_AUTH).getValue();
|
||||
final SSLContextService sslContextService = validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
||||
|
||||
if (sslContextService != null && StringUtils.isBlank(clientAuth)) {
|
||||
results.add(new ValidationResult.Builder()
|
||||
.explanation("Client Auth must be provided when using TLS/SSL")
|
||||
.valid(false).subject("Client Auth").build());
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
public void onScheduled(final ProcessContext context) throws IOException {
|
||||
this.port = context.getProperty(PORT).evaluateAttributeExpressions().asInteger();
|
||||
|
||||
final int readTimeout = context.getProperty(READ_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
|
||||
final int maxSocketBufferSize = context.getProperty(MAX_SOCKET_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
final int maxConnections = context.getProperty(MAX_CONNECTIONS).asInteger();
|
||||
final RecordReaderFactory recordReaderFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
|
||||
|
||||
// if the Network Interface Property wasn't provided then a null InetAddress will indicate to bind to all interfaces
|
||||
final InetAddress nicAddress;
|
||||
final String nicAddressStr = context.getProperty(NETWORK_INTF_NAME).evaluateAttributeExpressions().getValue();
|
||||
if (!StringUtils.isEmpty(nicAddressStr)) {
|
||||
NetworkInterface netIF = NetworkInterface.getByName(nicAddressStr);
|
||||
nicAddress = netIF.getInetAddresses().nextElement();
|
||||
} else {
|
||||
nicAddress = null;
|
||||
}
|
||||
|
||||
SSLContext sslContext = null;
|
||||
ClientAuth clientAuth = null;
|
||||
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
||||
if (sslContextService != null) {
|
||||
final String clientAuthValue = context.getProperty(CLIENT_AUTH).getValue();
|
||||
sslContext = sslContextService.createContext();
|
||||
clientAuth = ClientAuth.valueOf(clientAuthValue);
|
||||
}
|
||||
|
||||
// create a ServerSocketChannel in non-blocking mode and bind to the given address and port
|
||||
final ServerSocketChannel serverSocketChannel = ServerSocketChannel.open();
|
||||
serverSocketChannel.configureBlocking(false);
|
||||
serverSocketChannel.bind(new InetSocketAddress(nicAddress, port));
|
||||
|
||||
this.dispatcher = new SocketChannelRecordReaderDispatcher(serverSocketChannel, sslContext, clientAuth, readTimeout,
|
||||
maxSocketBufferSize, maxConnections, recordReaderFactory, socketReaders, getLogger());
|
||||
|
||||
// start a thread to run the dispatcher
|
||||
final Thread readerThread = new Thread(dispatcher);
|
||||
readerThread.setName(getClass().getName() + " [" + getIdentifier() + "]");
|
||||
readerThread.setDaemon(true);
|
||||
readerThread.start();
|
||||
}
|
||||
|
||||
public int getListeningPort() {
|
||||
return dispatcher.getPort();
|
||||
}
|
||||
|
||||
@OnStopped
|
||||
public void onStopped() {
|
||||
if (dispatcher != null) {
|
||||
dispatcher.close();
|
||||
dispatcher = null;
|
||||
}
|
||||
|
||||
SocketChannelRecordReader socketRecordReader;
|
||||
while ((socketRecordReader = socketReaders.poll()) != null) {
|
||||
try {
|
||||
socketRecordReader.close();
|
||||
} catch (Exception e) {
|
||||
getLogger().error("Couldn't close {}", socketRecordReader, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
final SocketChannelRecordReader socketRecordReader = pollForSocketRecordReader();
|
||||
if (socketRecordReader == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (socketRecordReader.isClosed()) {
|
||||
getLogger().warn("Unable to read records from {}, socket already closed", getRemoteAddress(socketRecordReader));
|
||||
IOUtils.closeQuietly(socketRecordReader); // still need to call close so the overall count is decremented
|
||||
return;
|
||||
}
|
||||
|
||||
final int recordBatchSize = context.getProperty(RECORD_BATCH_SIZE).asInteger();
|
||||
final String readerErrorHandling = context.getProperty(READER_ERROR_HANDLING_STRATEGY).getValue();
|
||||
final RecordSetWriterFactory recordSetWriterFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
|
||||
|
||||
// synchronize to ensure there are no stale values in the underlying SocketChannel
|
||||
synchronized (socketRecordReader) {
|
||||
FlowFile flowFile = session.create();
|
||||
try {
|
||||
// lazily creating the record reader here
|
||||
RecordReader recordReader = socketRecordReader.getRecordReader();
|
||||
if (recordReader == null) {
|
||||
recordReader = socketRecordReader.createRecordReader(getLogger());
|
||||
}
|
||||
|
||||
Record record;
|
||||
try {
|
||||
record = recordReader.nextRecord();
|
||||
} catch (final Exception e) {
|
||||
boolean timeout = false;
|
||||
|
||||
// some of the underlying record libraries wrap the real exception in RuntimeException, so check each
|
||||
// throwable (starting with the current one) to see if its a SocketTimeoutException
|
||||
Throwable cause = e;
|
||||
while (cause != null) {
|
||||
if (cause instanceof SocketTimeoutException) {
|
||||
timeout = true;
|
||||
break;
|
||||
}
|
||||
cause = cause.getCause();
|
||||
}
|
||||
|
||||
if (timeout) {
|
||||
getLogger().debug("Timeout reading records, will try again later", e);
|
||||
socketReaders.offer(socketRecordReader);
|
||||
session.remove(flowFile);
|
||||
return;
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
if (record == null) {
|
||||
getLogger().debug("No records available from {}, closing connection", getRemoteAddress(socketRecordReader));
|
||||
IOUtils.closeQuietly(socketRecordReader);
|
||||
session.remove(flowFile);
|
||||
return;
|
||||
}
|
||||
|
||||
String mimeType = null;
|
||||
WriteResult writeResult = null;
|
||||
|
||||
final RecordSchema recordSchema = recordSetWriterFactory.getSchema(Collections.EMPTY_MAP, record.getSchema());
|
||||
try (final OutputStream out = session.write(flowFile);
|
||||
final RecordSetWriter recordWriter = recordSetWriterFactory.createWriter(getLogger(), recordSchema, out, flowFile)) {
|
||||
|
||||
// start the record set and write the first record from above
|
||||
recordWriter.beginRecordSet();
|
||||
writeResult = recordWriter.write(record);
|
||||
|
||||
while (record != null && writeResult.getRecordCount() < recordBatchSize) {
|
||||
// handle a read failure according to the strategy selected...
|
||||
// if discarding then bounce to the outer catch block which will close the connection and remove the flow file
|
||||
// if keeping then null out the record to break out of the loop, which will transfer what we have and close the connection
|
||||
try {
|
||||
record = recordReader.nextRecord();
|
||||
} catch (final SocketTimeoutException ste) {
|
||||
getLogger().debug("Timeout reading records, will try again later", ste);
|
||||
break;
|
||||
} catch (final Exception e) {
|
||||
if (ERROR_HANDLING_DISCARD.getValue().equals(readerErrorHandling)) {
|
||||
throw e;
|
||||
} else {
|
||||
record = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (record != null) {
|
||||
writeResult = recordWriter.write(record);
|
||||
}
|
||||
}
|
||||
|
||||
writeResult = recordWriter.finishRecordSet();
|
||||
recordWriter.flush();
|
||||
mimeType = recordWriter.getMimeType();
|
||||
}
|
||||
|
||||
// if we didn't write any records then we need to remove the flow file
|
||||
if (writeResult.getRecordCount() <= 0) {
|
||||
getLogger().debug("Removing flow file, no records were written");
|
||||
session.remove(flowFile);
|
||||
} else {
|
||||
final String sender = getRemoteAddress(socketRecordReader);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>(writeResult.getAttributes());
|
||||
attributes.put(CoreAttributes.MIME_TYPE.key(), mimeType);
|
||||
attributes.put("tcp.sender", sender);
|
||||
attributes.put("tcp.port", String.valueOf(port));
|
||||
attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
|
||||
addClientCertificateAttributes(attributes, socketRecordReader);
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
final String senderHost = sender.startsWith("/") && sender.length() > 1 ? sender.substring(1) : sender;
|
||||
final String transitUri = new StringBuilder().append("tcp").append("://").append(senderHost).append(":").append(port).toString();
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri);
|
||||
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
}
|
||||
|
||||
getLogger().debug("Re-queuing connection for further processing...");
|
||||
socketReaders.offer(socketRecordReader);
|
||||
|
||||
} catch (Exception e) {
|
||||
getLogger().error("Error processing records", e);
|
||||
IOUtils.closeQuietly(socketRecordReader);
|
||||
session.remove(flowFile);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private SocketChannelRecordReader pollForSocketRecordReader() {
|
||||
try {
|
||||
return socketReaders.poll(POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private String getRemoteAddress(final SocketChannelRecordReader socketChannelRecordReader) {
|
||||
return socketChannelRecordReader.getRemoteAddress() == null ? "null" : socketChannelRecordReader.getRemoteAddress().toString();
|
||||
}
|
||||
|
||||
private void addClientCertificateAttributes(final Map<String, String> attributes, final SocketChannelRecordReader socketRecordReader) {
|
||||
if (socketRecordReader instanceof final SSLSocketChannelRecordReader sslSocketRecordReader) {
|
||||
final SSLSession sslSession = sslSocketRecordReader.getSession();
|
||||
try {
|
||||
final Certificate[] certificates = sslSession.getPeerCertificates();
|
||||
if (certificates.length > 0) {
|
||||
final X509Certificate certificate = (X509Certificate) certificates[0];
|
||||
final PrincipalFormatter principalFormatter = StandardPrincipalFormatter.getInstance();
|
||||
attributes.put(CLIENT_CERTIFICATE_SUBJECT_DN_ATTRIBUTE, principalFormatter.getSubject(certificate));
|
||||
attributes.put(CLIENT_CERTIFICATE_ISSUER_DN_ATTRIBUTE, principalFormatter.getIssuer(certificate));
|
||||
}
|
||||
} catch (final SSLPeerUnverifiedException peerUnverifiedException) {
|
||||
getLogger().debug("Remote Peer [{}] not verified: client certificates not provided",
|
||||
socketRecordReader.getRemoteAddress(), peerUnverifiedException);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,394 +0,0 @@
|
|||
/*
|
||||
* 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.standard;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import com.fluenda.parcefone.event.CEFHandlingException;
|
||||
import com.fluenda.parcefone.event.CommonEvent;
|
||||
import com.fluenda.parcefone.event.MacAddress;
|
||||
import com.fluenda.parcefone.parser.CEFParser;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
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.components.Validator;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.migration.PropertyConfiguration;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.InetAddress;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"logs", "cef", "attributes", "system", "event", "message"})
|
||||
@CapabilityDescription("Parses the contents of a CEF formatted message and adds attributes to the FlowFile for " +
|
||||
"headers and extensions of the parts of the CEF message.\n" +
|
||||
"Note: This Processor expects CEF messages WITHOUT the syslog headers (i.e. starting at \"CEF:0\"")
|
||||
@WritesAttributes({@WritesAttribute(attribute = "cef.header.version", description = "The version of the CEF message."),
|
||||
@WritesAttribute(attribute = "cef.header.deviceVendor", description = "The Device Vendor of the CEF message."),
|
||||
@WritesAttribute(attribute = "cef.header.deviceProduct", description = "The Device Product of the CEF message."),
|
||||
@WritesAttribute(attribute = "cef.header.deviceVersion", description = "The Device Version of the CEF message."),
|
||||
@WritesAttribute(attribute = "cef.header.deviceEventClassId", description = "The Device Event Class ID of the CEF message."),
|
||||
@WritesAttribute(attribute = "cef.header.name", description = "The name of the CEF message."),
|
||||
@WritesAttribute(attribute = "cef.header.severity", description = "The severity of the CEF message."),
|
||||
@WritesAttribute(attribute = "cef.extension.*", description = "The key and value generated by the parsing of the message.")})
|
||||
@SeeAlso({ParseSyslog.class})
|
||||
|
||||
public class ParseCEF extends AbstractProcessor {
|
||||
|
||||
// There should be no date format other than internationally agreed formats...
|
||||
// flowfile-attributes uses Java 8 time to parse data (as Date objects are not timezoned)
|
||||
private final static DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
|
||||
|
||||
// for some reason Jackson doesnt seem to be able to use DateTieFormater
|
||||
// so we use a DateFormat to format within flowfile-content
|
||||
private final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
|
||||
|
||||
|
||||
// add a TZ object to be used by flowfile-attribute routine
|
||||
private String tzId = null;
|
||||
|
||||
// Add serializer and mapper
|
||||
private static final ObjectMapper mapper = new ObjectMapper();
|
||||
|
||||
public static final String DESTINATION_CONTENT = "flowfile-content";
|
||||
public static final String DESTINATION_ATTRIBUTES = "flowfile-attribute";
|
||||
public static final PropertyDescriptor FIELDS_DESTINATION = new PropertyDescriptor.Builder()
|
||||
.name("FIELDS_DESTINATION")
|
||||
.displayName("Parsed fields destination")
|
||||
.description(
|
||||
"Indicates whether the results of the CEF parser are written " +
|
||||
"to the FlowFile content or a FlowFile attribute; if using " + DESTINATION_ATTRIBUTES +
|
||||
"attribute, fields will be populated as attributes. " +
|
||||
"If set to " + DESTINATION_CONTENT + ", the CEF extension field will be converted into " +
|
||||
"a flat JSON object.")
|
||||
.required(true)
|
||||
.allowableValues(DESTINATION_CONTENT, DESTINATION_ATTRIBUTES)
|
||||
.defaultValue(DESTINATION_CONTENT)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor APPEND_RAW_MESSAGE_TO_JSON = new PropertyDescriptor.Builder()
|
||||
.name("APPEND_RAW_MESSAGE_TO_JSON")
|
||||
.displayName("Append raw message to JSON")
|
||||
.description("When using flowfile-content (i.e. JSON output), add the original CEF message to " +
|
||||
"the resulting JSON object. The original message is added as a string to _raw.")
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.required(true)
|
||||
.defaultValue("true")
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor INCLUDE_CUSTOM_EXTENSIONS = new PropertyDescriptor.Builder()
|
||||
.name("INCLUDE_CUSTOM_EXTENSIONS")
|
||||
.displayName("Include custom extensions")
|
||||
.description("If set to true, custom extensions (not specified in the CEF specifications) will be "
|
||||
+ "included in the generated data/attributes.")
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.required(true)
|
||||
.defaultValue("false")
|
||||
.allowableValues("true", "false")
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor ACCEPT_EMPTY_EXTENSIONS = new PropertyDescriptor.Builder()
|
||||
.name("ACCEPT_EMPTY_EXTENSIONS")
|
||||
.displayName("Accept empty extensions")
|
||||
.description("If set to true, empty extensions will be accepted and will be associated to a null value.")
|
||||
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
|
||||
.required(true)
|
||||
.defaultValue("false")
|
||||
.allowableValues("true", "false")
|
||||
.build();
|
||||
|
||||
public static final String UTC = "UTC";
|
||||
public static final String LOCAL_TZ = "Local Timezone (system Default)";
|
||||
public static final PropertyDescriptor TIME_REPRESENTATION = new PropertyDescriptor.Builder()
|
||||
.name("TIME_REPRESENTATION")
|
||||
.displayName("Timezone")
|
||||
.description("Timezone to be used when representing date fields. UTC will convert all " +
|
||||
"dates to UTC, while Local Timezone will convert them to the timezone used by NiFi.")
|
||||
.allowableValues(UTC, LOCAL_TZ)
|
||||
.required(true)
|
||||
.defaultValue(LOCAL_TZ)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DATETIME_REPRESENTATION = new PropertyDescriptor.Builder()
|
||||
.name("DATETIME_REPRESENTATION")
|
||||
.displayName("DateTime Locale")
|
||||
.description("The IETF BCP 47 representation of the Locale to be used when parsing date " +
|
||||
"fields with long or short month names (e.g. may <en-US> vs. mai. <fr-FR>. The default" +
|
||||
"value is generally safe. Only change if having issues parsing CEF messages")
|
||||
.required(true)
|
||||
.addValidator(new ValidateLocale())
|
||||
.defaultValue("en-US")
|
||||
.build();
|
||||
|
||||
static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("Any FlowFile that could not be parsed as a CEF message will be transferred to this Relationship without any attributes being added")
|
||||
.build();
|
||||
static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("Any FlowFile that is successfully parsed as a CEF message will be transferred to this Relationship.")
|
||||
.build();
|
||||
|
||||
@Override
|
||||
public void migrateProperties(final PropertyConfiguration propertyConfiguration) {
|
||||
// Data Validation disabled due to ParCEFone dependency on javax.validation incompatibility with current Jakarta libraries
|
||||
propertyConfiguration.removeProperty("VALIDATE_DATA");
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
final List<PropertyDescriptor>properties = new ArrayList<>();
|
||||
properties.add(FIELDS_DESTINATION);
|
||||
properties.add(APPEND_RAW_MESSAGE_TO_JSON);
|
||||
properties.add(INCLUDE_CUSTOM_EXTENSIONS);
|
||||
properties.add(ACCEPT_EMPTY_EXTENSIONS);
|
||||
properties.add(TIME_REPRESENTATION);
|
||||
properties.add(DATETIME_REPRESENTATION);
|
||||
return properties;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
final Set<Relationship> relationships = new HashSet<>();
|
||||
relationships.add(REL_FAILURE);
|
||||
relationships.add(REL_SUCCESS);
|
||||
return relationships;
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
public void OnScheduled(final ProcessContext context) {
|
||||
|
||||
// Configure jackson mapper before spawning onTriggers
|
||||
final SimpleModule module = new SimpleModule()
|
||||
.addSerializer(MacAddress.class, new MacAddressToStringSerializer());
|
||||
mapper.registerModule(module);
|
||||
mapper.setDateFormat(this.simpleDateFormat);
|
||||
|
||||
switch (context.getProperty(TIME_REPRESENTATION).getValue()) {
|
||||
case LOCAL_TZ:
|
||||
// set the mapper TZ to local TZ
|
||||
mapper.setTimeZone(TimeZone.getDefault());
|
||||
tzId = TimeZone.getDefault().getID();
|
||||
break;
|
||||
case UTC:
|
||||
// set the mapper TZ to local TZ
|
||||
mapper.setTimeZone(TimeZone.getTimeZone(UTC));
|
||||
tzId = UTC;
|
||||
break;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
FlowFile flowFile = session.get();
|
||||
if (flowFile == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final CEFParser parser = new CEFParser();
|
||||
|
||||
final byte[] buffer = new byte[(int) flowFile.getSize()];
|
||||
session.read(flowFile, new InputStreamCallback() {
|
||||
@Override
|
||||
public void process(final InputStream in) throws IOException {
|
||||
StreamUtils.fillBuffer(in, buffer);
|
||||
}
|
||||
});
|
||||
|
||||
CommonEvent event;
|
||||
|
||||
try {
|
||||
// parcefoneLocale defaults to en_US, so this should not fail. But we force failure in case the custom
|
||||
// validator failed to identify an invalid Locale
|
||||
final Locale parcefoneLocale = Locale.forLanguageTag(context.getProperty(DATETIME_REPRESENTATION).getValue());
|
||||
final boolean acceptEmptyExtensions = context.getProperty(ACCEPT_EMPTY_EXTENSIONS).asBoolean();
|
||||
event = parser.parse(buffer, false, acceptEmptyExtensions, parcefoneLocale);
|
||||
|
||||
} catch (Exception e) {
|
||||
// This should never trigger but adding in here as a fencing mechanism to
|
||||
// address possible ParCEFone bugs.
|
||||
getLogger().error("CEF Parsing Failed: {}", flowFile, e);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
// ParCEFone returns null every time it cannot parse an
|
||||
// event, so we test
|
||||
if (event == null) {
|
||||
getLogger().error("Failed to parse {} as a CEF message: it does not conform to the CEF standard; routing to failure", flowFile);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
final String destination = context.getProperty(FIELDS_DESTINATION).getValue();
|
||||
final boolean includeCustomExtensions = context.getProperty(INCLUDE_CUSTOM_EXTENSIONS).asBoolean();
|
||||
|
||||
switch (destination) {
|
||||
case DESTINATION_ATTRIBUTES:
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
||||
// Process KVs of the Header field
|
||||
for (Map.Entry<String, Object> entry : event.getHeader().entrySet()) {
|
||||
attributes.put("cef.header." + entry.getKey(), prettyResult(entry.getValue(), tzId));
|
||||
}
|
||||
|
||||
// Process KVs composing the Extension field
|
||||
for (Map.Entry<String, Object> entry : event.getExtension(true, includeCustomExtensions).entrySet()) {
|
||||
attributes.put("cef.extension." + entry.getKey(), prettyResult(entry.getValue(), tzId));
|
||||
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
}
|
||||
break;
|
||||
|
||||
case DESTINATION_CONTENT:
|
||||
|
||||
ObjectNode results = mapper.createObjectNode();
|
||||
|
||||
// Add two JSON objects containing one CEF field each
|
||||
results.set("header", mapper.valueToTree(event.getHeader()));
|
||||
results.set("extension", mapper.valueToTree(event.getExtension(true, includeCustomExtensions)));
|
||||
|
||||
// Add the original content to original CEF content
|
||||
// to the resulting JSON
|
||||
if (context.getProperty(APPEND_RAW_MESSAGE_TO_JSON).asBoolean()) {
|
||||
results.set("_raw", mapper.valueToTree(new String(buffer)));
|
||||
}
|
||||
|
||||
flowFile = session.write(flowFile, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(OutputStream out) throws IOException {
|
||||
try (OutputStream outputStream = new BufferedOutputStream(out)) {
|
||||
outputStream.write(mapper.writeValueAsBytes(results));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
// Adjust the FlowFile mime.type attribute
|
||||
flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), "application/json");
|
||||
|
||||
// Update the provenance for good measure
|
||||
session.getProvenanceReporter().modifyContent(flowFile, "Replaced content with parsed CEF fields and values");
|
||||
break;
|
||||
}
|
||||
|
||||
// whatever the parsing stratgy, ready to transfer to success and commit
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
} catch (CEFHandlingException e) {
|
||||
// The flowfile has failed parsing & validation, routing to failure and committing
|
||||
getLogger().error("Reading CEF Event Failed: {}", flowFile, e);
|
||||
// Create a provenance event recording the routing to failure
|
||||
session.getProvenanceReporter().route(flowFile, REL_FAILURE);
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
}
|
||||
}
|
||||
|
||||
private String prettyResult(Object entryValue, String tzID) {
|
||||
|
||||
if (entryValue instanceof InetAddress ) {
|
||||
return ((InetAddress) entryValue).getHostAddress();
|
||||
} else if (entryValue instanceof Date) {
|
||||
ZonedDateTime zdt = ZonedDateTime.from(((Date) entryValue).toInstant().atZone(ZoneId.of(tzID)));
|
||||
return (String.valueOf(zdt.format(dateTimeFormatter)));
|
||||
} else {
|
||||
return String.valueOf(entryValue);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// Serialize MacAddress as plain string
|
||||
private class MacAddressToStringSerializer extends JsonSerializer<MacAddress> {
|
||||
|
||||
@Override
|
||||
public void serialize(MacAddress macAddress,
|
||||
JsonGenerator jsonGenerator,
|
||||
SerializerProvider serializerProvider)
|
||||
throws IOException {
|
||||
jsonGenerator.writeObject(macAddress.toString());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected static class ValidateLocale implements Validator {
|
||||
@Override
|
||||
public ValidationResult validate(String subject, String input, ValidationContext context) {
|
||||
if (null == input || input.isEmpty()) {
|
||||
return new ValidationResult.Builder().subject(subject).input(input).valid(false)
|
||||
.explanation(subject + " cannot be empty").build();
|
||||
}
|
||||
|
||||
final Locale testLocale = Locale.forLanguageTag(input);
|
||||
|
||||
// Check if the provided Locale is valid by checking against the empty locale string
|
||||
if ("".equals(testLocale.toString())) {
|
||||
// Locale matches the "null" locale so it is treated as invalid
|
||||
return new ValidationResult.Builder().subject(subject).input(input).valid(false)
|
||||
.explanation(input + " is not a valid locale format.").build();
|
||||
} else {
|
||||
return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,7 +25,6 @@ import org.apache.nifi.annotation.behavior.SupportsBatching;
|
|||
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;
|
||||
|
@ -83,7 +82,6 @@ import static java.util.stream.Collectors.toList;
|
|||
import static org.apache.nifi.processor.util.pattern.ExceptionHandler.createOnError;
|
||||
|
||||
@SupportsBatching
|
||||
@SeeAlso(ConvertJSONToSQL.class)
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"sql", "put", "rdbms", "database", "update", "insert", "relational"})
|
||||
@CapabilityDescription("Executes a SQL UPDATE or INSERT command. The content of an incoming FlowFile is expected to be the SQL command "
|
||||
|
|
|
@ -1,42 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.event;
|
||||
|
||||
import org.apache.nifi.event.transport.message.ByteArrayMessage;
|
||||
|
||||
/**
|
||||
* A RELP message which adds a transaction number and command to the ByteArrayMessage.
|
||||
*/
|
||||
public class RELPMessage extends ByteArrayMessage {
|
||||
|
||||
private final long txnr;
|
||||
private final String command;
|
||||
|
||||
public RELPMessage(final String sender, final byte[] data, final long txnr, final String command) {
|
||||
super(data, sender);
|
||||
this.txnr = txnr;
|
||||
this.command = command;
|
||||
}
|
||||
|
||||
public long getTxnr() {
|
||||
return txnr;
|
||||
}
|
||||
|
||||
public String getCommand() {
|
||||
return command;
|
||||
}
|
||||
}
|
|
@ -1,35 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.event;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.event.NetworkEventFactory;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* An EventFactory implementation to create RELPEvents.
|
||||
*/
|
||||
public class RELPMessageFactory implements NetworkEventFactory<RELPMessage> {
|
||||
|
||||
@Override
|
||||
public RELPMessage create(final byte[] data, final Map<String, String> metadata) {
|
||||
final long txnr = Long.valueOf(metadata.get(RELPMetadata.TXNR_KEY));
|
||||
final String command = metadata.get(RELPMetadata.COMMAND_KEY);
|
||||
final String sender = metadata.get(RELPMetadata.SENDER_KEY);
|
||||
return new RELPMessage(sender, data, txnr, command);
|
||||
}
|
||||
}
|
|
@ -1,28 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.event;
|
||||
|
||||
/**
|
||||
* Metadata keys for RELP.
|
||||
*/
|
||||
public interface RELPMetadata {
|
||||
|
||||
String TXNR_KEY = "relp.txnr";
|
||||
String COMMAND_KEY = "relp.command";
|
||||
String SENDER_KEY = "sender";
|
||||
|
||||
}
|
|
@ -1,202 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
|
||||
/**
|
||||
* Decodes a RELP frame by maintaining a state based on each byte that has been processed. This class
|
||||
* should not be shared by multiple threads.
|
||||
*/
|
||||
public class RELPDecoder {
|
||||
|
||||
static final Logger logger = LoggerFactory.getLogger(RELPDecoder.class);
|
||||
|
||||
private RELPFrame.Builder frameBuilder;
|
||||
private RELPState currState = RELPState.TXNR;
|
||||
|
||||
private final Charset charset;
|
||||
private final ByteArrayOutputStream currBytes;
|
||||
|
||||
/**
|
||||
* @param charset the charset to decode bytes from the RELP frame
|
||||
*/
|
||||
public RELPDecoder(final Charset charset) {
|
||||
this(charset, new ByteArrayOutputStream(4096));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param charset the charset to decode bytes from the RELP frame
|
||||
*/
|
||||
public RELPDecoder(final Charset charset, final int bufferSize) {
|
||||
this(charset, new ByteArrayOutputStream(bufferSize));
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param charset the charset to decode bytes from the RELP frame
|
||||
* @param buffer a buffer to use while processing the bytes
|
||||
*/
|
||||
public RELPDecoder(final Charset charset, final ByteArrayOutputStream buffer) {
|
||||
this.charset = charset;
|
||||
this.currBytes = buffer;
|
||||
this.frameBuilder = new RELPFrame.Builder();
|
||||
}
|
||||
|
||||
/**
|
||||
* Resets this decoder back to it's initial state.
|
||||
*/
|
||||
public void reset() {
|
||||
frameBuilder = new RELPFrame.Builder();
|
||||
currState = RELPState.TXNR;
|
||||
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 RELPFrameException {
|
||||
try {
|
||||
switch (currState) {
|
||||
case TXNR:
|
||||
processTXNR(currByte);
|
||||
break;
|
||||
case COMMAND:
|
||||
processCOMMAND(currByte);
|
||||
break;
|
||||
case LENGTH:
|
||||
processLENGTH(currByte);
|
||||
// if jumped from length to trailer we need to return true here
|
||||
// because there might not be another byte to process
|
||||
if (currState == RELPState.TRAILER) {
|
||||
return true;
|
||||
}
|
||||
break;
|
||||
case DATA:
|
||||
processDATA(currByte);
|
||||
break;
|
||||
case TRAILER:
|
||||
return true;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return false;
|
||||
} catch (Exception e) {
|
||||
throw new RELPFrameException("Error decoding RELP 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 RELPFrame that was decoded
|
||||
*/
|
||||
public RELPFrame getFrame() throws RELPFrameException {
|
||||
if (currState != RELPState.TRAILER) {
|
||||
throw new RELPFrameException("Must be at the trailer of a frame");
|
||||
}
|
||||
|
||||
try {
|
||||
final RELPFrame frame = frameBuilder.build();
|
||||
processTRAILER(RELPFrame.DELIMITER);
|
||||
return frame;
|
||||
} catch (Exception e) {
|
||||
throw new RELPFrameException("Error decoding RELP frame: " + e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void processTXNR(final byte b) {
|
||||
if (b == RELPFrame.SEPARATOR) {
|
||||
if (currBytes.size() > 0) {
|
||||
final long txnr = Long.parseLong(new String(currBytes.toByteArray(), charset));
|
||||
frameBuilder.txnr(txnr);
|
||||
logger.debug("Transaction number is {}", txnr);
|
||||
|
||||
currBytes.reset();
|
||||
currState = RELPState.COMMAND;
|
||||
}
|
||||
} else {
|
||||
currBytes.write(b);
|
||||
}
|
||||
}
|
||||
|
||||
private void processCOMMAND(final byte b) {
|
||||
if (b == RELPFrame.SEPARATOR) {
|
||||
final String command = new String(currBytes.toByteArray(), charset);
|
||||
frameBuilder.command(command);
|
||||
logger.debug("Command is {}", command);
|
||||
|
||||
currBytes.reset();
|
||||
currState = RELPState.LENGTH;
|
||||
} else {
|
||||
currBytes.write(b);
|
||||
}
|
||||
}
|
||||
|
||||
private void processLENGTH(final byte b) {
|
||||
if (b == RELPFrame.SEPARATOR || (currBytes.size() > 0 && b == RELPFrame.DELIMITER)) {
|
||||
final int dataLength = Integer.parseInt(new String(currBytes.toByteArray(), charset));
|
||||
frameBuilder.dataLength(dataLength);
|
||||
logger.debug("Length is {}", dataLength);
|
||||
|
||||
currBytes.reset();
|
||||
|
||||
// if at a separator then data is going to follow, but if at a separator there is no data
|
||||
if (b == RELPFrame.SEPARATOR) {
|
||||
currState = RELPState.DATA;
|
||||
} else {
|
||||
frameBuilder.data(new byte[0]);
|
||||
currState = RELPState.TRAILER;
|
||||
}
|
||||
} else {
|
||||
currBytes.write(b);
|
||||
}
|
||||
}
|
||||
|
||||
private void processDATA(final byte b) {
|
||||
currBytes.write(b);
|
||||
logger.trace("Data size is {}", currBytes.size());
|
||||
|
||||
if (currBytes.size() >= frameBuilder.dataLength) {
|
||||
final byte[] data = currBytes.toByteArray();
|
||||
frameBuilder.data(data);
|
||||
logger.debug("Reached expected data size of {}", frameBuilder.dataLength);
|
||||
|
||||
currBytes.reset();
|
||||
currState = RELPState.TRAILER;
|
||||
}
|
||||
}
|
||||
|
||||
private void processTRAILER(final byte b) {
|
||||
if (b != RELPFrame.DELIMITER) {
|
||||
logger.warn("Expected RELP trailing LF, but found another byte");
|
||||
}
|
||||
currBytes.reset();
|
||||
frameBuilder.reset();
|
||||
currState = RELPState.TXNR;
|
||||
}
|
||||
|
||||
}
|
|
@ -1,66 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
|
||||
/**
|
||||
* Encodes a RELPFrame into raw bytes using the given charset.
|
||||
*/
|
||||
public class RELPEncoder {
|
||||
|
||||
private final Charset charset;
|
||||
|
||||
public RELPEncoder(final Charset charset) {
|
||||
this.charset = charset;
|
||||
}
|
||||
|
||||
public Charset getCharset() {
|
||||
return charset;
|
||||
}
|
||||
|
||||
public byte[] encode(final RELPFrame frame) {
|
||||
final ByteArrayOutputStream buffer = new ByteArrayOutputStream();
|
||||
|
||||
// write transaction number followed by separator
|
||||
byte[] txnr = String.format("%s", frame.getTxnr()).getBytes(charset);
|
||||
buffer.write(txnr, 0, txnr.length);
|
||||
buffer.write(RELPFrame.SEPARATOR);
|
||||
|
||||
// write the command followed by separator
|
||||
byte[] command = frame.getCommand().getBytes(charset);
|
||||
buffer.write(command, 0, command.length);
|
||||
buffer.write(RELPFrame.SEPARATOR);
|
||||
|
||||
// write the data length
|
||||
byte[] dataLength = String.format("%s", frame.getDataLength()).getBytes(charset);
|
||||
buffer.write(dataLength, 0, dataLength.length);
|
||||
|
||||
// if data to write then put a separator and write the data
|
||||
if (frame.getDataLength() > 0) {
|
||||
buffer.write(RELPFrame.SEPARATOR);
|
||||
buffer.write(frame.getData(), 0, frame.getDataLength());
|
||||
}
|
||||
|
||||
// write the end of the frame
|
||||
buffer.write(RELPFrame.DELIMITER);
|
||||
|
||||
return buffer.toByteArray();
|
||||
}
|
||||
|
||||
}
|
|
@ -1,110 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
/**
|
||||
* A RELP frame received from a channel.
|
||||
*/
|
||||
public class RELPFrame {
|
||||
|
||||
public static final byte DELIMITER = 10;
|
||||
public static final byte SEPARATOR = 32;
|
||||
|
||||
private final long txnr;
|
||||
private final int dataLength;
|
||||
private final String command;
|
||||
private final byte[] data;
|
||||
|
||||
private RELPFrame(final Builder builder) {
|
||||
this.txnr = builder.txnr;
|
||||
this.dataLength = builder.dataLength;
|
||||
this.command = builder.command;
|
||||
this.data = builder.data == null ? new byte[0] : builder.data;
|
||||
|
||||
if (txnr < 0 || dataLength < 0 || command == null || StringUtils.isBlank(command)
|
||||
|| data == null || dataLength != data.length) {
|
||||
throw new RELPFrameException("Invalid Frame");
|
||||
}
|
||||
}
|
||||
|
||||
public long getTxnr() {
|
||||
return txnr;
|
||||
}
|
||||
|
||||
public int getDataLength() {
|
||||
return dataLength;
|
||||
}
|
||||
|
||||
public String getCommand() {
|
||||
return command;
|
||||
}
|
||||
|
||||
//NOTE: consider making a copy here if we want to be truly be immutable
|
||||
public byte[] getData() {
|
||||
return data;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Builder for a RELPFrame.
|
||||
*/
|
||||
public static class Builder {
|
||||
|
||||
long txnr;
|
||||
int dataLength;
|
||||
String command;
|
||||
byte[] data;
|
||||
|
||||
public Builder() {
|
||||
reset();
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
txnr = -1;
|
||||
dataLength = -1;
|
||||
command = null;
|
||||
data = null;
|
||||
}
|
||||
|
||||
public Builder txnr(final long txnr) {
|
||||
this.txnr = txnr;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder dataLength(final int dataLength) {
|
||||
this.dataLength = dataLength;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder command(final String command) {
|
||||
this.command = command;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder data(final byte[] data) {
|
||||
this.data = data;
|
||||
return this;
|
||||
}
|
||||
|
||||
public RELPFrame build() {
|
||||
return new RELPFrame(this);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,95 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.ByteToMessageDecoder;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactoryUtil;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMessageFactory;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMetadata;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Decode RELP message bytes into a RELPMessage
|
||||
*/
|
||||
public class RELPFrameDecoder extends ByteToMessageDecoder {
|
||||
|
||||
private Charset charset;
|
||||
private RELPDecoder decoder;
|
||||
private final ComponentLog logger;
|
||||
private final RELPEncoder encoder;
|
||||
private final RELPMessageFactory eventFactory;
|
||||
|
||||
static final String CMD_OPEN = "open";
|
||||
static final String CMD_CLOSE = "close";
|
||||
|
||||
public RELPFrameDecoder(final ComponentLog logger, final Charset charset) {
|
||||
this.charset = charset;
|
||||
this.logger = logger;
|
||||
this.encoder = new RELPEncoder(charset);
|
||||
this.eventFactory = new RELPMessageFactory();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void decode(final ChannelHandlerContext ctx, final ByteBuf in, final List<Object> out) throws Exception {
|
||||
final int total = in.readableBytes();
|
||||
final String senderSocket = ctx.channel().remoteAddress().toString();
|
||||
this.decoder = new RELPDecoder(charset, total);
|
||||
|
||||
// go through the buffer parsing the RELP command
|
||||
for (int i = 0; i < total; i++) {
|
||||
byte currByte = in.readByte();
|
||||
// if we found the end of a frame, handle the frame and mark the buffer
|
||||
if (decoder.process(currByte)) {
|
||||
final RELPFrame frame = decoder.getFrame();
|
||||
|
||||
logger.debug("Received RELP frame with transaction {} and command {}",
|
||||
frame.getTxnr(), frame.getCommand());
|
||||
handle(frame, ctx, senderSocket, out);
|
||||
in.markReaderIndex();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void handle(final RELPFrame frame, final ChannelHandlerContext ctx, final String sender, final List<Object> out) {
|
||||
// respond to open and close commands immediately, create and queue an event for everything else
|
||||
if (CMD_OPEN.equals(frame.getCommand())) {
|
||||
Map<String, String> offers = RELPResponse.parseOffers(frame.getData(), charset);
|
||||
ChannelResponse response = new RELPChannelResponse(encoder, RELPResponse.open(frame.getTxnr(), offers));
|
||||
ctx.writeAndFlush(Unpooled.wrappedBuffer(response.toByteArray()));
|
||||
} else if (CMD_CLOSE.equals(frame.getCommand())) {
|
||||
ChannelResponse response = new RELPChannelResponse(encoder, RELPResponse.ok(frame.getTxnr()));
|
||||
ctx.writeAndFlush(Unpooled.wrappedBuffer(response.toByteArray()));
|
||||
ctx.close();
|
||||
} else {
|
||||
final Map<String, String> metadata = EventFactoryUtil.createMapWithSender(sender);
|
||||
metadata.put(RELPMetadata.TXNR_KEY, String.valueOf(frame.getTxnr()));
|
||||
metadata.put(RELPMetadata.COMMAND_KEY, frame.getCommand());
|
||||
metadata.put(RELPMetadata.SENDER_KEY, sender);
|
||||
out.add(eventFactory.create(frame.getData(), metadata));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,32 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
/**
|
||||
* Represents an error encountered when decoding RELP frames.
|
||||
*/
|
||||
public class RELPFrameException extends RuntimeException {
|
||||
|
||||
public RELPFrameException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public RELPFrameException(String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,58 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.SimpleChannelInboundHandler;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMessage;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Decode data received into a RELPMessage
|
||||
*/
|
||||
@ChannelHandler.Sharable
|
||||
public class RELPMessageChannelHandler extends SimpleChannelInboundHandler<RELPMessage> {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(RELPMessageChannelHandler.class);
|
||||
private final BlockingQueue<RELPMessage> events;
|
||||
private final RELPEncoder encoder;
|
||||
|
||||
public RELPMessageChannelHandler(BlockingQueue<RELPMessage> events, final Charset charset) {
|
||||
this.events = events;
|
||||
this.encoder = new RELPEncoder(charset);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void channelRead0(ChannelHandlerContext ctx, RELPMessage msg) {
|
||||
LOGGER.debug("RELP Message Received Length [{}] Remote Address [{}] ", msg.getMessage().length, msg.getSender());
|
||||
if (events.offer(msg)) {
|
||||
LOGGER.debug("Event Queued: RELP Message Sender [{}] Transaction Number [{}]", msg.getSender(), msg.getTxnr());
|
||||
ctx.writeAndFlush(Unpooled.wrappedBuffer(new RELPChannelResponse(encoder, RELPResponse.ok(msg.getTxnr())).toByteArray()));
|
||||
} else {
|
||||
LOGGER.debug("Event Queue Full: Failed RELP Message Sender [{}] Transaction Number [{}]", msg.getSender(), msg.getTxnr());
|
||||
ctx.writeAndFlush(Unpooled.wrappedBuffer(new RELPChannelResponse(encoder, RELPResponse.serverFullError(msg.getTxnr())).toByteArray()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,43 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.MessageToMessageEncoder;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Message encoder for a RELPResponse
|
||||
*/
|
||||
@ChannelHandler.Sharable
|
||||
public class RELPResponseEncoder extends MessageToMessageEncoder<RELPResponse> {
|
||||
|
||||
private Charset charset;
|
||||
|
||||
public RELPResponseEncoder(final Charset charset) {
|
||||
this.charset = charset;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void encode(ChannelHandlerContext context, RELPResponse event, List<Object> out) throws Exception {
|
||||
out.add(new RELPEncoder(charset).encode(event.toFrame(charset)));
|
||||
}
|
||||
}
|
|
@ -1,30 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
/**
|
||||
* The parts of a RELP frame.
|
||||
*/
|
||||
public enum RELPState {
|
||||
|
||||
TXNR,
|
||||
COMMAND,
|
||||
LENGTH,
|
||||
DATA,
|
||||
TRAILER
|
||||
|
||||
}
|
|
@ -1,60 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.handler;
|
||||
|
||||
import org.apache.nifi.event.transport.configuration.TransportProtocol;
|
||||
import org.apache.nifi.event.transport.netty.NettyEventServerFactory;
|
||||
import org.apache.nifi.event.transport.netty.channel.LogExceptionChannelHandler;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMessage;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrameDecoder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPMessageChannelHandler;
|
||||
|
||||
import java.net.InetAddress;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Netty Event Server Factory implementation for RELP Messages
|
||||
*/
|
||||
public class RELPMessageServerFactory extends NettyEventServerFactory {
|
||||
|
||||
/**
|
||||
* RELP Message Server Factory to receive RELP messages
|
||||
* @param log Component Log
|
||||
* @param address Server Address
|
||||
* @param port Server Port Number
|
||||
* @param charset Charset to use when decoding RELP messages
|
||||
* @param events Blocking Queue for events received
|
||||
*/
|
||||
public RELPMessageServerFactory(final ComponentLog log,
|
||||
final InetAddress address,
|
||||
final int port,
|
||||
final Charset charset,
|
||||
final BlockingQueue<RELPMessage> events) {
|
||||
super(address, port, TransportProtocol.TCP);
|
||||
final LogExceptionChannelHandler logExceptionChannelHandler = new LogExceptionChannelHandler(log);
|
||||
final RELPMessageChannelHandler relpChannelHandler = new RELPMessageChannelHandler(events, charset);
|
||||
|
||||
setHandlerSupplier(() -> Arrays.asList(
|
||||
new RELPFrameDecoder(log, charset),
|
||||
relpChannelHandler,
|
||||
logExceptionChannelHandler
|
||||
));
|
||||
}
|
||||
}
|
|
@ -1,42 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.response;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPEncoder;
|
||||
|
||||
/**
|
||||
* Creates a RELPFrame for the provided response and returns the encoded frame.
|
||||
*/
|
||||
public class RELPChannelResponse implements ChannelResponse {
|
||||
|
||||
private final RELPEncoder encoder;
|
||||
private final RELPResponse response;
|
||||
|
||||
public RELPChannelResponse(final RELPEncoder encoder, final RELPResponse response) {
|
||||
this.encoder = encoder;
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] toByteArray() {
|
||||
final RELPFrame frame = response.toFrame(encoder.getCharset());
|
||||
return encoder.encode(frame);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,176 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.response;
|
||||
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* The data portion of a RELPFrame for a response:
|
||||
*
|
||||
* RSP-CODE [SP HUMANMSG] LF [CMDDATA]
|
||||
*
|
||||
*/
|
||||
public class RELPResponse {
|
||||
|
||||
public static final int OK = 200;
|
||||
public static final int ERROR = 500;
|
||||
|
||||
public static final String RSP_CMD = "rsp";
|
||||
|
||||
private final long txnr;
|
||||
private final int code;
|
||||
private final String message;
|
||||
private final String data;
|
||||
|
||||
public RELPResponse(final long txnr, final int code) {
|
||||
this(txnr, code, null, null);
|
||||
}
|
||||
|
||||
public RELPResponse(final long txnr, final int code, final String message, final String data) {
|
||||
this.txnr = txnr;
|
||||
this.code = code;
|
||||
this.message = message;
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a RELPFrame where the data portion will contain this response.
|
||||
*
|
||||
* @param charset the character set to encode the response
|
||||
*
|
||||
* @return a RELPFrame for for this response
|
||||
*/
|
||||
public RELPFrame toFrame(final Charset charset) {
|
||||
final StringBuilder builder = new StringBuilder();
|
||||
builder.append(code);
|
||||
|
||||
if (message != null && !message.isEmpty()) {
|
||||
builder.append((char) RELPFrame.SEPARATOR);
|
||||
builder.append(message);
|
||||
}
|
||||
|
||||
if (data != null) {
|
||||
builder.append((char) RELPFrame.DELIMITER);
|
||||
builder.append(data);
|
||||
}
|
||||
|
||||
final byte[] data = builder.toString().getBytes(charset);
|
||||
|
||||
return new RELPFrame.Builder()
|
||||
.txnr(txnr).command(RSP_CMD)
|
||||
.dataLength(data.length).data(data)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to create a response to an open request.
|
||||
*
|
||||
* @param txnr the transaction number of the open request
|
||||
* @param offers the accepted offers
|
||||
*
|
||||
* @return the RELPResponse for the given open request
|
||||
*/
|
||||
public static RELPResponse open(final long txnr, final Map<String, String> offers) {
|
||||
int i = 0;
|
||||
final StringBuilder sb = new StringBuilder();
|
||||
for (final Map.Entry<String, String> entry : offers.entrySet()) {
|
||||
if (i > 0) {
|
||||
sb.append((char) RELPFrame.DELIMITER);
|
||||
}
|
||||
|
||||
sb.append(entry.getKey());
|
||||
|
||||
if (entry.getValue() != null) {
|
||||
sb.append('=');
|
||||
sb.append(entry.getValue());
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
return new RELPResponse(txnr, OK, "OK", sb.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to create a default "OK" response.
|
||||
*
|
||||
* @param txnr the transaction number being responded to
|
||||
*
|
||||
* @return a RELPResponse with a 200 code and a message of "OK"
|
||||
*/
|
||||
public static RELPResponse ok(final long txnr) {
|
||||
return new RELPResponse(txnr, OK, "OK", null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to create a default "ERROR" response.
|
||||
*
|
||||
* @param txnr the transaction number being responded to
|
||||
*
|
||||
* @return a RELPResponse with a 500 code and a message of "ERROR"
|
||||
*/
|
||||
public static RELPResponse error(final long txnr) {
|
||||
return error(txnr, "ERROR");
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to create a default "ERROR" response if the server event queue is full.
|
||||
*
|
||||
* @param txnr the transaction number being responded to
|
||||
*
|
||||
* @return a RELPResponse with a 500 code and a message of "SERVER FULL"
|
||||
*/
|
||||
public static RELPResponse serverFullError(final long txnr) {
|
||||
return error(txnr, "SERVER FULL");
|
||||
}
|
||||
|
||||
private static RELPResponse error(final long txnr, final String message) {
|
||||
return new RELPResponse(txnr, ERROR, message, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses the provided data into a Map of offers.
|
||||
*
|
||||
* @param data the data portion of a RELPFrame for an "open" command
|
||||
* @param charset the charset to decode the data
|
||||
*
|
||||
* @return a Map of offers, or an empty Map if no data is provided
|
||||
*/
|
||||
public static Map<String, String> parseOffers(final byte[] data, final Charset charset) {
|
||||
final Map<String, String> offers = new HashMap<>();
|
||||
if (data == null || data.length == 0) {
|
||||
return offers;
|
||||
}
|
||||
|
||||
final String dataStr = new String(data, charset);
|
||||
final String[] splits = dataStr.split("[" + (char) RELPFrame.DELIMITER + "]");
|
||||
|
||||
for (final String split : splits) {
|
||||
final String[] fields = split.split( "=", 2);
|
||||
if (fields.length > 1 ) {
|
||||
offers.put(fields[0], fields[1]);
|
||||
} else {
|
||||
offers.put(fields[0], fields[0]);
|
||||
}
|
||||
}
|
||||
|
||||
return offers;
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@ org.apache.nifi.processors.standard.CalculateRecordStats
|
|||
org.apache.nifi.processors.standard.CompressContent
|
||||
org.apache.nifi.processors.standard.ControlRate
|
||||
org.apache.nifi.processors.standard.ConvertCharacterSet
|
||||
org.apache.nifi.processors.standard.ConvertJSONToSQL
|
||||
org.apache.nifi.processors.standard.ConvertRecord
|
||||
org.apache.nifi.processors.standard.CountText
|
||||
org.apache.nifi.processors.standard.CryptographicHashContent
|
||||
|
@ -62,10 +61,8 @@ org.apache.nifi.processors.standard.JoinEnrichment
|
|||
org.apache.nifi.processors.standard.ListDatabaseTables
|
||||
org.apache.nifi.processors.standard.ListenFTP
|
||||
org.apache.nifi.processors.standard.ListenHTTP
|
||||
org.apache.nifi.processors.standard.ListenRELP
|
||||
org.apache.nifi.processors.standard.ListenSyslog
|
||||
org.apache.nifi.processors.standard.ListenTCP
|
||||
org.apache.nifi.processors.standard.ListenTCPRecord
|
||||
org.apache.nifi.processors.standard.ListenUDP
|
||||
org.apache.nifi.processors.standard.ListenUDPRecord
|
||||
org.apache.nifi.processors.standard.ListFile
|
||||
|
@ -81,7 +78,6 @@ org.apache.nifi.processors.standard.ModifyBytes
|
|||
org.apache.nifi.processors.standard.MonitorActivity
|
||||
org.apache.nifi.processors.standard.Notify
|
||||
org.apache.nifi.processors.standard.PackageFlowFile
|
||||
org.apache.nifi.processors.standard.ParseCEF
|
||||
org.apache.nifi.processors.standard.ParseSyslog
|
||||
org.apache.nifi.processors.standard.ParseSyslog5424
|
||||
org.apache.nifi.processors.standard.PartitionRecord
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1,277 +0,0 @@
|
|||
/*
|
||||
* 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.standard;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.ftpserver.ssl.ClientAuth;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.event.transport.EventSender;
|
||||
import org.apache.nifi.event.transport.configuration.ShutdownQuietPeriod;
|
||||
import org.apache.nifi.event.transport.configuration.ShutdownTimeout;
|
||||
import org.apache.nifi.event.transport.configuration.TransportProtocol;
|
||||
import org.apache.nifi.event.transport.netty.ByteArrayNettyEventSenderFactory;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.util.listen.ListenerProperties;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMessage;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPEncoder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.ssl.RestrictedSSLContextService;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.apache.nifi.web.util.ssl.SslContextUtils;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class TestListenRELP {
|
||||
|
||||
public static final String OPEN_FRAME_DATA = "relp_version=0\nrelp_software=librelp,1.2.7,http://librelp.adiscon.com\ncommands=syslog";
|
||||
public static final String RELP_FRAME_DATA = "this is a relp message here";
|
||||
|
||||
private static final String LOCALHOST = "localhost";
|
||||
private static final Charset CHARSET = StandardCharsets.US_ASCII;
|
||||
private static final Duration SENDER_TIMEOUT = Duration.ofSeconds(10);
|
||||
|
||||
static final RELPFrame OPEN_FRAME = new RELPFrame.Builder()
|
||||
.txnr(1)
|
||||
.command("open")
|
||||
.dataLength(OPEN_FRAME_DATA.length())
|
||||
.data(OPEN_FRAME_DATA.getBytes(CHARSET))
|
||||
.build();
|
||||
|
||||
static final RELPFrame RELP_FRAME = new RELPFrame.Builder()
|
||||
.txnr(2)
|
||||
.command("syslog")
|
||||
.dataLength(RELP_FRAME_DATA.length())
|
||||
.data(RELP_FRAME_DATA.getBytes(CHARSET))
|
||||
.build();
|
||||
|
||||
static final RELPFrame CLOSE_FRAME = new RELPFrame.Builder()
|
||||
.txnr(3)
|
||||
.command("close")
|
||||
.dataLength(0)
|
||||
.data(new byte[0])
|
||||
.build();
|
||||
|
||||
@Mock
|
||||
private RestrictedSSLContextService sslContextService;
|
||||
|
||||
private RELPEncoder encoder;
|
||||
|
||||
private TestRunner runner;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() {
|
||||
encoder = new RELPEncoder(CHARSET);
|
||||
ListenRELP mockRELP = new MockListenRELP();
|
||||
runner = TestRunners.newTestRunner(mockRELP);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void shutdown() {
|
||||
runner.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRELPFramesAreReceivedSuccessfully() throws Exception {
|
||||
final int relpFrames = 5;
|
||||
final List<RELPFrame> frames = getFrames(relpFrames);
|
||||
|
||||
// three RELP frames should be transferred
|
||||
run(frames, relpFrames, null);
|
||||
|
||||
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
|
||||
assertNotNull(events);
|
||||
assertEquals(relpFrames, events.size());
|
||||
|
||||
final ProvenanceEventRecord event = events.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, event.getEventType());
|
||||
assertTrue(event.getTransitUri().toLowerCase().startsWith("relp"), "transit uri must be set and start with proper protocol");
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenRELP.REL_SUCCESS);
|
||||
assertEquals(relpFrames, mockFlowFiles.size());
|
||||
|
||||
final MockFlowFile mockFlowFile = mockFlowFiles.get(0);
|
||||
assertEquals(String.valueOf(RELP_FRAME.getTxnr()), mockFlowFile.getAttribute(ListenRELP.RELPAttributes.TXNR.key()));
|
||||
assertEquals(RELP_FRAME.getCommand(), mockFlowFile.getAttribute(ListenRELP.RELPAttributes.COMMAND.key()));
|
||||
assertFalse(StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.PORT.key())));
|
||||
assertFalse(StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.SENDER.key())));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRELPFramesAreReceivedSuccessfullyWhenBatched() throws Exception {
|
||||
|
||||
runner.setProperty(ListenerProperties.MAX_BATCH_SIZE, "5");
|
||||
|
||||
final int relpFrames = 3;
|
||||
final List<RELPFrame> frames = getFrames(relpFrames);
|
||||
|
||||
// one relp frame should be transferred since we are batching
|
||||
final int expectedFlowFiles = 1;
|
||||
run(frames, expectedFlowFiles, null);
|
||||
|
||||
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
|
||||
assertNotNull(events);
|
||||
assertEquals(expectedFlowFiles, events.size());
|
||||
|
||||
final ProvenanceEventRecord event = events.get(0);
|
||||
assertEquals(ProvenanceEventType.RECEIVE, event.getEventType());
|
||||
assertTrue(event.getTransitUri().toLowerCase().startsWith("relp"), "transit uri must be set and start with proper protocol");
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenRELP.REL_SUCCESS);
|
||||
assertEquals(expectedFlowFiles, mockFlowFiles.size());
|
||||
|
||||
final MockFlowFile mockFlowFile = mockFlowFiles.get(0);
|
||||
assertEquals(RELP_FRAME.getCommand(), mockFlowFile.getAttribute(ListenRELP.RELPAttributes.COMMAND.key()));
|
||||
assertFalse(StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.PORT.key())));
|
||||
assertFalse(StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.SENDER.key())));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunMutualTls() throws Exception {
|
||||
final String serviceIdentifier = SSLContextService.class.getName();
|
||||
when(sslContextService.getIdentifier()).thenReturn(serviceIdentifier);
|
||||
final SSLContext sslContext = SslContextUtils.createKeyStoreSslContext();
|
||||
when(sslContextService.createContext()).thenReturn(sslContext);
|
||||
runner.addControllerService(serviceIdentifier, sslContextService);
|
||||
runner.enableControllerService(sslContextService);
|
||||
|
||||
runner.setProperty(ListenRELP.SSL_CONTEXT_SERVICE, serviceIdentifier);
|
||||
runner.setProperty(ListenRELP.CLIENT_AUTH, ClientAuth.NONE.name());
|
||||
|
||||
final int relpFrames = 3;
|
||||
final List<RELPFrame> frames = getFrames(relpFrames);
|
||||
run(frames, relpFrames, sslContext);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBatchingWithDifferentSenders() {
|
||||
String sender1 = "/192.168.1.50:55000";
|
||||
String sender2 = "/192.168.1.50:55001";
|
||||
String sender3 = "/192.168.1.50:55002";
|
||||
|
||||
final List<RELPMessage> mockEvents = new ArrayList<>();
|
||||
mockEvents.add(new RELPMessage(sender1, RELP_FRAME.getData(), RELP_FRAME.getTxnr(), RELP_FRAME.getCommand()));
|
||||
mockEvents.add(new RELPMessage(sender1, RELP_FRAME.getData(), RELP_FRAME.getTxnr(), RELP_FRAME.getCommand()));
|
||||
mockEvents.add(new RELPMessage(sender1, RELP_FRAME.getData(), RELP_FRAME.getTxnr(), RELP_FRAME.getCommand()));
|
||||
mockEvents.add(new RELPMessage(sender2, RELP_FRAME.getData(), RELP_FRAME.getTxnr(), RELP_FRAME.getCommand()));
|
||||
mockEvents.add(new RELPMessage(sender3, RELP_FRAME.getData(), RELP_FRAME.getTxnr(), RELP_FRAME.getCommand()));
|
||||
mockEvents.add(new RELPMessage(sender3, RELP_FRAME.getData(), RELP_FRAME.getTxnr(), RELP_FRAME.getCommand()));
|
||||
|
||||
MockListenRELP mockListenRELP = new MockListenRELP(mockEvents);
|
||||
runner = TestRunners.newTestRunner(mockListenRELP);
|
||||
runner.setProperty(ListenerProperties.PORT, "0");
|
||||
runner.setProperty(ListenerProperties.MAX_BATCH_SIZE, "10");
|
||||
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(ListenRELP.REL_SUCCESS, 3);
|
||||
runner.shutdown();
|
||||
}
|
||||
|
||||
private void run(final List<RELPFrame> frames, final int flowFiles, final SSLContext sslContext) throws Exception {
|
||||
runner.setProperty(ListenerProperties.PORT, "0");
|
||||
runner.run(1, false, true);
|
||||
|
||||
final int port = ((ListenRELP) runner.getProcessor()).getListeningPort();
|
||||
|
||||
// Run Processor and start Dispatcher without shutting down
|
||||
final byte[] relpMessages = getRELPMessages(frames);
|
||||
sendMessages(port, relpMessages, sslContext);
|
||||
runner.run(flowFiles, false, false);
|
||||
runner.assertTransferCount(ListenRELP.REL_SUCCESS, flowFiles);
|
||||
}
|
||||
|
||||
private byte[] getRELPMessages(final List<RELPFrame> frames) throws IOException {
|
||||
|
||||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
|
||||
for (final RELPFrame frame : frames) {
|
||||
final byte[] encodedFrame = encoder.encode(frame);
|
||||
outputStream.write(encodedFrame);
|
||||
outputStream.flush();
|
||||
}
|
||||
|
||||
return outputStream.toByteArray();
|
||||
}
|
||||
|
||||
private List<RELPFrame> getFrames(final int relpFrames) {
|
||||
final List<RELPFrame> frames = new ArrayList<>();
|
||||
frames.add(OPEN_FRAME);
|
||||
|
||||
for (int i = 0; i < relpFrames; i++) {
|
||||
frames.add(RELP_FRAME);
|
||||
}
|
||||
|
||||
frames.add(CLOSE_FRAME);
|
||||
return frames;
|
||||
}
|
||||
|
||||
private void sendMessages(final int port, final byte[] relpMessages, final SSLContext sslContext) throws Exception {
|
||||
final ByteArrayNettyEventSenderFactory eventSenderFactory = new ByteArrayNettyEventSenderFactory(runner.getLogger(), LOCALHOST, port, TransportProtocol.TCP);
|
||||
eventSenderFactory.setShutdownQuietPeriod(ShutdownQuietPeriod.QUICK.getDuration());
|
||||
eventSenderFactory.setShutdownTimeout(ShutdownTimeout.QUICK.getDuration());
|
||||
if (sslContext != null) {
|
||||
eventSenderFactory.setSslContext(sslContext);
|
||||
}
|
||||
|
||||
eventSenderFactory.setTimeout(SENDER_TIMEOUT);
|
||||
try (final EventSender<byte[]> eventSender = eventSenderFactory.getEventSender()) {
|
||||
eventSender.sendEvent(relpMessages);
|
||||
}
|
||||
}
|
||||
|
||||
private static class MockListenRELP extends ListenRELP {
|
||||
private final List<RELPMessage> mockEvents;
|
||||
|
||||
public MockListenRELP() {
|
||||
this.mockEvents = new ArrayList<>();
|
||||
}
|
||||
|
||||
public MockListenRELP(List<RELPMessage> mockEvents) {
|
||||
this.mockEvents = mockEvents;
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
@Override
|
||||
public void onScheduled(ProcessContext context) throws IOException {
|
||||
super.onScheduled(context);
|
||||
events.addAll(mockEvents);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,267 +0,0 @@
|
|||
/*
|
||||
* 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.standard;
|
||||
|
||||
import org.apache.nifi.json.JsonTreeReader;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.schema.access.SchemaAccessUtils;
|
||||
import org.apache.nifi.security.util.ClientAuth;
|
||||
import org.apache.nifi.security.util.TlsException;
|
||||
import org.apache.nifi.serialization.RecordReaderFactory;
|
||||
import org.apache.nifi.serialization.RecordSetWriterFactory;
|
||||
import org.apache.nifi.serialization.record.MockRecordWriter;
|
||||
import org.apache.nifi.ssl.RestrictedSSLContextService;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.apache.nifi.web.util.ssl.SslContextUtils;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.mockito.Mockito;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.Socket;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestListenTCPRecord {
|
||||
static final String SCHEMA_TEXT = "{\n" +
|
||||
" \"name\": \"syslogRecord\",\n" +
|
||||
" \"namespace\": \"nifi\",\n" +
|
||||
" \"type\": \"record\",\n" +
|
||||
" \"fields\": [\n" +
|
||||
" { \"name\": \"timestamp\", \"type\": \"string\" },\n" +
|
||||
" { \"name\": \"logsource\", \"type\": \"string\" },\n" +
|
||||
" { \"name\": \"message\", \"type\": \"string\" }\n" +
|
||||
" ]\n" +
|
||||
"}";
|
||||
|
||||
static final String DATA = "[" +
|
||||
"{\"timestamp\" : \"123456789\", \"logsource\" : \"syslog\", \"message\" : \"This is a test 1\"}," +
|
||||
"{\"timestamp\" : \"123456789\", \"logsource\" : \"syslog\", \"message\" : \"This is a test 2\"}," +
|
||||
"{\"timestamp\" : \"123456789\", \"logsource\" : \"syslog\", \"message\" : \"This is a test 3\"}" +
|
||||
"]";
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(TestListenTCPRecord.class);
|
||||
|
||||
private static final long TEST_TIMEOUT = 30;
|
||||
|
||||
private static final String LOCALHOST = "localhost";
|
||||
|
||||
private static final String SSL_CONTEXT_IDENTIFIER = SSLContextService.class.getName();
|
||||
|
||||
private static SSLContext keyStoreSslContext;
|
||||
|
||||
private static SSLContext trustStoreSslContext;
|
||||
|
||||
private TestRunner runner;
|
||||
|
||||
@BeforeAll
|
||||
public static void configureServices() throws TlsException {
|
||||
keyStoreSslContext = SslContextUtils.createKeyStoreSslContext();
|
||||
trustStoreSslContext = SslContextUtils.createTrustStoreSslContext();
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws InitializationException {
|
||||
runner = TestRunners.newTestRunner(ListenTCPRecord.class);
|
||||
|
||||
final String readerId = "record-reader";
|
||||
final RecordReaderFactory readerFactory = new JsonTreeReader();
|
||||
runner.addControllerService(readerId, readerFactory);
|
||||
runner.setProperty(readerFactory, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY.getValue());
|
||||
runner.setProperty(readerFactory, SchemaAccessUtils.SCHEMA_TEXT, SCHEMA_TEXT);
|
||||
runner.enableControllerService(readerFactory);
|
||||
|
||||
final String writerId = "record-writer";
|
||||
final RecordSetWriterFactory writerFactory = new MockRecordWriter("timestamp, logsource, message");
|
||||
runner.addControllerService(writerId, writerFactory);
|
||||
runner.enableControllerService(writerFactory);
|
||||
|
||||
runner.setProperty(ListenTCPRecord.RECORD_READER, readerId);
|
||||
runner.setProperty(ListenTCPRecord.RECORD_WRITER, writerId);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCustomValidate() throws InitializationException {
|
||||
runner.setProperty(ListenTCPRecord.PORT, "1");
|
||||
runner.assertValid();
|
||||
|
||||
enableSslContextService(keyStoreSslContext);
|
||||
runner.setProperty(ListenTCPRecord.CLIENT_AUTH, "");
|
||||
runner.assertNotValid();
|
||||
|
||||
runner.setProperty(ListenTCPRecord.CLIENT_AUTH, ClientAuth.REQUIRED.name());
|
||||
runner.assertValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
@Timeout(TEST_TIMEOUT)
|
||||
public void testRunOneRecordPerFlowFile() throws IOException, InterruptedException {
|
||||
runner.setProperty(ListenTCPRecord.RECORD_BATCH_SIZE, "1");
|
||||
|
||||
run(3, null);
|
||||
|
||||
List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenTCPRecord.REL_SUCCESS);
|
||||
for (int i = 0; i < mockFlowFiles.size(); i++) {
|
||||
final MockFlowFile flowFile = mockFlowFiles.get(i);
|
||||
flowFile.assertAttributeEquals("record.count", "1");
|
||||
|
||||
final String content = new String(flowFile.toByteArray(), StandardCharsets.UTF_8);
|
||||
assertNotNull(content);
|
||||
assertTrue(content.contains("This is a test " + (i + 1)));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Timeout(TEST_TIMEOUT)
|
||||
public void testRunMultipleRecordsPerFlowFileLessThanBatchSize() throws IOException, InterruptedException {
|
||||
runner.setProperty(ListenTCPRecord.RECORD_BATCH_SIZE, "5");
|
||||
|
||||
run(1, null);
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenTCPRecord.REL_SUCCESS);
|
||||
assertEquals(1, mockFlowFiles.size());
|
||||
|
||||
final MockFlowFile flowFile = mockFlowFiles.get(0);
|
||||
flowFile.assertAttributeEquals("record.count", "3");
|
||||
|
||||
final String content = new String(flowFile.toByteArray(), StandardCharsets.UTF_8);
|
||||
assertNotNull(content);
|
||||
assertTrue(content.contains("This is a test " + 1));
|
||||
assertTrue(content.contains("This is a test " + 2));
|
||||
assertTrue(content.contains("This is a test " + 3));
|
||||
}
|
||||
|
||||
@Test
|
||||
@Timeout(TEST_TIMEOUT)
|
||||
public void testRunClientAuthRequired() throws InitializationException, IOException, InterruptedException {
|
||||
runner.setProperty(ListenTCPRecord.CLIENT_AUTH, ClientAuth.REQUIRED.name());
|
||||
enableSslContextService(keyStoreSslContext);
|
||||
|
||||
run(1, keyStoreSslContext);
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenTCPRecord.REL_SUCCESS);
|
||||
assertEquals(1, mockFlowFiles.size());
|
||||
|
||||
final String content = new String(mockFlowFiles.get(0).toByteArray(), StandardCharsets.UTF_8);
|
||||
assertNotNull(content);
|
||||
assertTrue(content.contains("This is a test " + 1));
|
||||
assertTrue(content.contains("This is a test " + 2));
|
||||
assertTrue(content.contains("This is a test " + 3));
|
||||
}
|
||||
|
||||
@Test
|
||||
@Timeout(TEST_TIMEOUT)
|
||||
public void testRunSSLClientDNsAddedAsAttributes() throws InitializationException, IOException, InterruptedException {
|
||||
runner.setProperty(ListenTCPRecord.CLIENT_AUTH, ClientAuth.REQUIRED.name());
|
||||
enableSslContextService(keyStoreSslContext);
|
||||
|
||||
run(1, keyStoreSslContext);
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenTCPRecord.REL_SUCCESS);
|
||||
assertEquals(1, mockFlowFiles.size());
|
||||
|
||||
final MockFlowFile flowFile = mockFlowFiles.get(0);
|
||||
flowFile.assertAttributeEquals("client.certificate.subject.dn", "CN=localhost");
|
||||
flowFile.assertAttributeEquals("client.certificate.issuer.dn", "CN=localhost");
|
||||
}
|
||||
|
||||
@Test
|
||||
@Timeout(TEST_TIMEOUT)
|
||||
public void testRunClientAuthNone() throws InitializationException, IOException, InterruptedException {
|
||||
runner.setProperty(ListenTCPRecord.CLIENT_AUTH, ClientAuth.NONE.name());
|
||||
enableSslContextService(keyStoreSslContext);
|
||||
|
||||
run(1, trustStoreSslContext);
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenTCPRecord.REL_SUCCESS);
|
||||
assertEquals(1, mockFlowFiles.size());
|
||||
|
||||
final String content = new String(mockFlowFiles.get(0).toByteArray(), StandardCharsets.UTF_8);
|
||||
assertNotNull(content);
|
||||
assertTrue(content.contains("This is a test " + 1));
|
||||
assertTrue(content.contains("This is a test " + 2));
|
||||
assertTrue(content.contains("This is a test " + 3));
|
||||
}
|
||||
|
||||
protected void run(final int expectedTransferred, final SSLContext sslContext) throws IOException, InterruptedException {
|
||||
runner.setProperty(ListenTCPRecord.PORT, "0");
|
||||
|
||||
// Run Processor and start listener without shutting down
|
||||
runner.run(1, false, true);
|
||||
final int port = ((ListenTCPRecord) runner.getProcessor()).getListeningPort();
|
||||
|
||||
final Thread thread = new Thread(() -> {
|
||||
try (final Socket socket = getSocket(port, sslContext)) {
|
||||
final OutputStream outputStream = socket.getOutputStream();
|
||||
outputStream.write(DATA.getBytes(StandardCharsets.UTF_8));
|
||||
outputStream.flush();
|
||||
/**
|
||||
* Once NIFI-12098 is resolved this sleep can be removed. Without the sleep
|
||||
* on some systems (Windows builds often) the build can fail and lockup indefinitely.
|
||||
*/
|
||||
Thread.sleep(3000);
|
||||
} catch (final IOException | InterruptedException e) {
|
||||
LOGGER.error("Failed Sending Records to Port [{}]", port, e);
|
||||
}
|
||||
});
|
||||
thread.start();
|
||||
|
||||
// Run Processor until success leveraging test method timeouts for failure status
|
||||
int iterations = 0;
|
||||
while (getSuccessCount() < expectedTransferred) {
|
||||
runner.run(1, false, false);
|
||||
iterations++;
|
||||
}
|
||||
LOGGER.info("Completed after iterations [{}]", iterations);
|
||||
}
|
||||
|
||||
private int getSuccessCount() {
|
||||
return runner.getFlowFilesForRelationship(ListenTCPRecord.REL_SUCCESS).size();
|
||||
}
|
||||
|
||||
private Socket getSocket(final int port, final SSLContext sslContext) throws IOException {
|
||||
final Socket socket;
|
||||
if (sslContext == null) {
|
||||
socket = new Socket(LOCALHOST, port);
|
||||
} else {
|
||||
socket = sslContext.getSocketFactory().createSocket(LOCALHOST, port);
|
||||
}
|
||||
return socket;
|
||||
}
|
||||
|
||||
private void enableSslContextService(final SSLContext sslContext) throws InitializationException {
|
||||
final RestrictedSSLContextService sslContextService = Mockito.mock(RestrictedSSLContextService.class);
|
||||
Mockito.when(sslContextService.getIdentifier()).thenReturn(SSL_CONTEXT_IDENTIFIER);
|
||||
Mockito.when(sslContextService.createContext()).thenReturn(sslContext);
|
||||
runner.addControllerService(SSL_CONTEXT_IDENTIFIER, sslContextService);
|
||||
runner.enableControllerService(sslContextService);
|
||||
runner.setProperty(ListenTCPRecord.SSL_CONTEXT_SERVICE, SSL_CONTEXT_IDENTIFIER);
|
||||
}
|
||||
}
|
|
@ -1,262 +0,0 @@
|
|||
/*
|
||||
* 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.standard;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Instant;
|
||||
import java.time.ZoneId;
|
||||
import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.util.Locale;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestParseCEF {
|
||||
private static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
|
||||
private static final ZonedDateTime DATE_TIME = Instant.ofEpochMilli(1423441663000L).atZone(ZoneId.systemDefault());
|
||||
|
||||
private final static String sample1 = "CEF:0|TestVendor|TestProduct|TestVersion|TestEventClassID|TestName|Low|" +
|
||||
// TimeStamp, String and Long
|
||||
"rt=Feb 09 2015 00:27:43 UTC cn3Label=Test Long cn3=9223372036854775807 " +
|
||||
// FloatPoint and MacAddress
|
||||
"cfp1=1.234 cfp1Label=Test FP Number smac=00:00:0c:07:ac:00 " +
|
||||
// IPv6 and String
|
||||
"c6a3=2001:cdba::3257:9652 c6a3Label=Test IPv6 cs1Label=Test String cs1=test test test chocolate " +
|
||||
// IPv4
|
||||
"destinationTranslatedAddress=123.123.123.123 " +
|
||||
// Date without TZ
|
||||
"deviceCustomDate1=Feb 06 2015 13:27:43 " +
|
||||
// Integer and IP Address (from v4)
|
||||
"dpt=1234 agt=123.123.0.124 dlat=40.366633";
|
||||
|
||||
private final static String sample2 = "CEF:0|TestVendor|TestProduct|TestVersion|TestEventClassID|TestName|Low|" +
|
||||
// TimeStamp, String and Long
|
||||
"rt=Feb 09 2015 00:27:43 UTC cn3Label=Test Long cn3=9223372036854775807 " +
|
||||
// FloatPoint and MacAddress
|
||||
"cfp1=1.234 cfp1Label=Test FP Number smac=00:00:0c:07:ac:00 " +
|
||||
// IPv6 and String
|
||||
"c6a3=2001:cdba::3257:9652 c6a3Label=Test IPv6 cs1Label=Test String cs1=test test test chocolate " +
|
||||
// IPv4
|
||||
"destinationTranslatedAddress=123.123.123.123 " +
|
||||
// Date without TZ
|
||||
"deviceCustomDate1=Feb 06 2015 13:27:43 " +
|
||||
// Integer and IP Address (from v4)
|
||||
"dpt=1234 agt=123.123.0.124 dlat=40.366633 " +
|
||||
// A JSON object inside one of CEF's custom Strings
|
||||
"cs2Label=JSON payload " +
|
||||
"cs2={\"test_test_test\": \"chocolate!\", \"what?!?\": \"Simple! test test test chocolate!\"}";
|
||||
|
||||
|
||||
@Test
|
||||
public void testInvalidMessage() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
|
||||
runner.enqueue("test test test chocolate\n".getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ParseCEF.REL_FAILURE, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessfulParseToAttributes() throws IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
|
||||
runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_ATTRIBUTES);
|
||||
runner.enqueue(sample1.getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
|
||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
|
||||
mff.assertAttributeEquals("cef.extension.rt", FORMATTER.format(DATE_TIME));
|
||||
mff.assertAttributeEquals("cef.extension.cn3Label", "Test Long");
|
||||
mff.assertAttributeEquals("cef.extension.cn3", "9223372036854775807");
|
||||
mff.assertAttributeEquals("cef.extension.cfp1", "1.234");
|
||||
mff.assertAttributeEquals("cef.extension.cfp1Label", "Test FP Number");
|
||||
mff.assertAttributeEquals("cef.extension.smac", "00:00:0c:07:ac:00");
|
||||
mff.assertAttributeEquals("cef.extension.c6a3", "2001:cdba:0:0:0:0:3257:9652");
|
||||
mff.assertAttributeEquals("cef.extension.c6a3Label", "Test IPv6");
|
||||
mff.assertAttributeEquals("cef.extension.cs1Label", "Test String");
|
||||
mff.assertAttributeEquals("cef.extension.cs1", "test test test chocolate");
|
||||
mff.assertAttributeEquals("cef.extension.destinationTranslatedAddress", "123.123.123.123");
|
||||
mff.assertContentEquals(sample1.getBytes());
|
||||
|
||||
mff.assertAttributeEquals("cef.extension.dpt", "1234");
|
||||
mff.assertAttributeEquals("cef.extension.agt", "123.123.0.124");
|
||||
mff.assertAttributeEquals("cef.extension.dlat", "40.366633");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessfulParseToContent() throws IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
|
||||
runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
|
||||
runner.enqueue(sample1.getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
|
||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
|
||||
|
||||
byte[] rawJson = mff.toByteArray();
|
||||
|
||||
JsonNode results = new ObjectMapper().readTree(rawJson);
|
||||
|
||||
JsonNode header = results.get("header");
|
||||
JsonNode extension = results.get("extension");
|
||||
|
||||
assertEquals("TestVendor", header.get("deviceVendor").asText());
|
||||
assertEquals(FORMATTER.format(DATE_TIME),
|
||||
extension.get("rt").asText());
|
||||
assertEquals("Test Long", extension.get("cn3Label").asText());
|
||||
assertEquals( 9223372036854775807L, extension.get("cn3").asLong());
|
||||
assertEquals(extension.get("cfp1").floatValue(), 1.234F);
|
||||
assertEquals("Test FP Number", extension.get("cfp1Label").asText());
|
||||
assertEquals("00:00:0c:07:ac:00", extension.get("smac").asText());
|
||||
assertEquals("2001:cdba:0:0:0:0:3257:9652", extension.get("c6a3").asText());
|
||||
assertEquals("Test IPv6", extension.get("c6a3Label").asText());
|
||||
assertEquals("123.123.123.123", extension.get("destinationTranslatedAddress").asText());
|
||||
assertEquals("Test String", extension.get("cs1Label").asText());
|
||||
assertEquals("test test test chocolate", extension.get("cs1").asText());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessfulParseToContentWhenCEFContainsJSON() throws IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
|
||||
runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
|
||||
runner.enqueue(sample2.getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
|
||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
|
||||
|
||||
byte[] rawJson = mff.toByteArray();
|
||||
|
||||
JsonNode results = new ObjectMapper().readTree(rawJson);
|
||||
|
||||
JsonNode header = results.get("header");
|
||||
JsonNode extension = results.get("extension");
|
||||
|
||||
assertEquals("TestVendor", header.get("deviceVendor").asText());
|
||||
assertEquals(FORMATTER.format(DATE_TIME),
|
||||
extension.get("rt").asText());
|
||||
assertEquals("Test Long", extension.get("cn3Label").asText());
|
||||
assertEquals( 9223372036854775807L, extension.get("cn3").asLong());
|
||||
assertEquals(extension.get("cfp1").floatValue(), 1.234F);
|
||||
assertEquals("Test FP Number", extension.get("cfp1Label").asText());
|
||||
assertEquals("00:00:0c:07:ac:00", extension.get("smac").asText());
|
||||
assertEquals("2001:cdba:0:0:0:0:3257:9652", extension.get("c6a3").asText());
|
||||
assertEquals("Test IPv6", extension.get("c6a3Label").asText());
|
||||
assertEquals("Test String", extension.get("cs1Label").asText());
|
||||
assertEquals("test test test chocolate", extension.get("cs1").asText());
|
||||
assertEquals("123.123.123.123", extension.get("destinationTranslatedAddress").asText());
|
||||
|
||||
JsonNode inner = new ObjectMapper().readTree(extension.get("cs2").asText());
|
||||
assertEquals("chocolate!", inner.get("test_test_test").asText());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCustomValidator() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
|
||||
runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
|
||||
runner.setProperty(ParseCEF.TIME_REPRESENTATION, ParseCEF.UTC);
|
||||
|
||||
|
||||
runner.setProperty(ParseCEF.DATETIME_REPRESENTATION, "SPANGLISH");
|
||||
runner.assertNotValid();
|
||||
|
||||
runner.setProperty(ParseCEF.DATETIME_REPRESENTATION, "en-US");
|
||||
runner.assertValid();
|
||||
|
||||
Locale[] availableLocales = Locale.getAvailableLocales();
|
||||
for (Locale listedLocale : availableLocales ) {
|
||||
if (!listedLocale.toString().isEmpty()) {
|
||||
String input = listedLocale.toLanguageTag();
|
||||
runner.setProperty(ParseCEF.DATETIME_REPRESENTATION, input );
|
||||
runner.assertValid();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIncludeCustomExtensions() throws Exception {
|
||||
String sample3 = "<159>Aug 09 08:56:28 8.8.8.8 CEF:0|x|Security|x.x.0|20|Transaction blocked|7| "
|
||||
+ "act=blocked app=https dvc=8.8.8.8 dst=8.8.8.8 dhost=www.flynas.com dpt=443 src=8.8.8.8 "
|
||||
+ "spt=53475 suser=x UserPath=LDAP://8.8.8.8 OU\\\\=1 - x x x x,OU\\\\=x x,DC\\\\=x,DC\\\\=com/x "
|
||||
+ "destinationTranslatedPort=36436 rt=1628488588000 in=65412 out=546 requestMethod=GET "
|
||||
+ "category=20 http_response=200 http_proxy_status_code=302 duration=13 "
|
||||
+ "requestClientApplication=Mozilla/5.0 (Windows NT 10.0; WOW64; Trident/7.0;) like Gecko reason=- "
|
||||
+ "cs1Label=Policy cs1=x x**x cs2Label=DynCat cs2=0 cs3Label=ContentType cs3=font/otf "
|
||||
+ "cn1Label=DispositionCode cn1=1047 cn2Label=ScanDuration cn2=13 "
|
||||
+ "request=https://www.flynas.com/css/fonts/GothamRounded-Book.otf URLRefer=https://www.flynas.com/en";
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
|
||||
runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
|
||||
runner.setProperty(ParseCEF.TIME_REPRESENTATION, ParseCEF.UTC);
|
||||
runner.setProperty(ParseCEF.INCLUDE_CUSTOM_EXTENSIONS, "true");
|
||||
runner.enqueue(sample3.getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
|
||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
|
||||
|
||||
byte[] rawJson = mff.toByteArray();
|
||||
|
||||
JsonNode results = new ObjectMapper().readTree(rawJson);
|
||||
|
||||
JsonNode extension = results.get("extension");
|
||||
assertEquals(200, extension.get("http_response").asInt());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAcceptEmptyExtensions() throws Exception {
|
||||
String sample3 = "CEF:0|TestVendor|TestProduct|TestVersion|TestEventClassID|TestName|Low|" +
|
||||
"rt=Feb 09 2015 00:27:43 UTC cn3Label= cn3= " +
|
||||
"cfp1=1.234 cfp1Label=Test FP Number smac=00:00:0c:07:ac:00 " +
|
||||
"c6a3=2001:cdba::3257:9652 c6a3Label=Test IPv6 cs1Label=Test String cs1=test test test chocolate " +
|
||||
"destinationTranslatedAddress=123.123.123.123 " +
|
||||
"deviceCustomDate1=Feb 06 2015 13:27:43 " +
|
||||
"dpt= agt= dlat=";
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ParseCEF());
|
||||
runner.setProperty(ParseCEF.FIELDS_DESTINATION, ParseCEF.DESTINATION_CONTENT);
|
||||
runner.setProperty(ParseCEF.TIME_REPRESENTATION, ParseCEF.UTC);
|
||||
runner.setProperty(ParseCEF.INCLUDE_CUSTOM_EXTENSIONS, "true");
|
||||
runner.setProperty(ParseCEF.ACCEPT_EMPTY_EXTENSIONS, "true");
|
||||
runner.enqueue(sample3.getBytes());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ParseCEF.REL_SUCCESS, 1);
|
||||
final MockFlowFile mff = runner.getFlowFilesForRelationship(ParseCEF.REL_SUCCESS).get(0);
|
||||
|
||||
byte[] rawJson = mff.toByteArray();
|
||||
|
||||
JsonNode results = new ObjectMapper().readTree(rawJson);
|
||||
|
||||
JsonNode extensions = results.get("extension");
|
||||
assertTrue(extensions.has("cn3"));
|
||||
assertTrue(extensions.get("cn3").isNull());
|
||||
|
||||
assertTrue(extensions.has("cn3Label"));
|
||||
assertTrue(extensions.get("cn3Label").asText().isEmpty());
|
||||
}
|
||||
}
|
||||
|
|
@ -1,103 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPEncoder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.Socket;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
public class RELPFrameProducer {
|
||||
|
||||
public static final String OPEN_FRAME_DATA = "relp_version=0\nrelp_software=librelp,1.2.7,http://librelp.adiscon.com\ncommands=syslog";
|
||||
|
||||
static final RELPFrame OPEN_FRAME = new RELPFrame.Builder()
|
||||
.txnr(1)
|
||||
.command("open")
|
||||
.dataLength(OPEN_FRAME_DATA.length())
|
||||
.data(OPEN_FRAME_DATA.getBytes(StandardCharsets.UTF_8))
|
||||
.build();
|
||||
|
||||
static final RELPFrame CLOSE_FRAME = new RELPFrame.Builder()
|
||||
.txnr(3)
|
||||
.command("close")
|
||||
.dataLength(0)
|
||||
.data(new byte[0])
|
||||
.build();
|
||||
|
||||
public static void main(String[] args) {
|
||||
if (args == null || args.length != 5) {
|
||||
System.err.println("USAGE: RELPFrameProducer <HOST> <PORT> <NUM_MSGS> <DELAY_INTERVAL> <DELAY_MILLIS>");
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
final String host = args[0];
|
||||
final int port = Integer.parseInt(args[1]);
|
||||
final int numMessages = Integer.parseInt(args[2]);
|
||||
final int delayInterval = Integer.parseInt(args[3]);
|
||||
final long delay = Long.parseLong(args[4]);
|
||||
|
||||
final RELPEncoder encoder = new RELPEncoder(StandardCharsets.UTF_8);
|
||||
|
||||
Socket socket = null;
|
||||
try {
|
||||
socket = new Socket(host, port);
|
||||
|
||||
try (final OutputStream out = new BufferedOutputStream(socket.getOutputStream())) {
|
||||
// send the open frame
|
||||
out.write(encoder.encode(OPEN_FRAME));
|
||||
|
||||
// send the specified number of syslog messages
|
||||
for (int i = 2; i < (numMessages + 2); i++) {
|
||||
final byte[] data = ("this is message # " + i).getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
final RELPFrame syslogFrame = new RELPFrame.Builder()
|
||||
.txnr(i)
|
||||
.command("syslog")
|
||||
.dataLength(data.length)
|
||||
.data(data)
|
||||
.build();
|
||||
|
||||
out.write(encoder.encode(syslogFrame));
|
||||
|
||||
if (i % delayInterval == 0) {
|
||||
System.out.println("Sent " + i + " messages");
|
||||
out.flush();
|
||||
Thread.sleep(delay);
|
||||
}
|
||||
}
|
||||
|
||||
// send the close frame
|
||||
out.write(encoder.encode(CLOSE_FRAME));
|
||||
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
} catch (final IOException e) {
|
||||
e.printStackTrace();
|
||||
} finally {
|
||||
IOUtils.closeQuietly(socket);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,99 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import io.netty.buffer.ByteBufOutputStream;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.embedded.EmbeddedChannel;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMessage;
|
||||
import org.apache.nifi.util.MockComponentLog;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
class RELPFrameDecoderTest {
|
||||
|
||||
final ComponentLog logger = new MockComponentLog(this.getClass().getSimpleName(), this);
|
||||
|
||||
public static final String OPEN_FRAME_DATA = "relp_version=0\nrelp_software=librelp,1.2.7,http://librelp.adiscon.com\ncommands=syslog";
|
||||
public static final String SYSLOG_FRAME_DATA = "this is a syslog message here";
|
||||
|
||||
static final RELPFrame OPEN_FRAME = new RELPFrame.Builder()
|
||||
.txnr(1)
|
||||
.command("open")
|
||||
.dataLength(OPEN_FRAME_DATA.length())
|
||||
.data(OPEN_FRAME_DATA.getBytes(StandardCharsets.UTF_8))
|
||||
.build();
|
||||
|
||||
static final RELPFrame SYSLOG_FRAME = new RELPFrame.Builder()
|
||||
.txnr(2)
|
||||
.command("syslog")
|
||||
.dataLength(SYSLOG_FRAME_DATA.length())
|
||||
.data(SYSLOG_FRAME_DATA.getBytes(StandardCharsets.UTF_8))
|
||||
.build();
|
||||
|
||||
static final RELPFrame CLOSE_FRAME = new RELPFrame.Builder()
|
||||
.txnr(3)
|
||||
.command("close")
|
||||
.dataLength(0)
|
||||
.data(new byte[0])
|
||||
.build();
|
||||
|
||||
@Test
|
||||
void testDecodeRELPEvents() throws IOException {
|
||||
final List<RELPFrame> frames = getFrames(5);
|
||||
ByteBufOutputStream eventBytes = new ByteBufOutputStream(Unpooled.buffer());
|
||||
sendFrames(frames, eventBytes);
|
||||
EmbeddedChannel channel = new EmbeddedChannel(new RELPFrameDecoder(logger, StandardCharsets.UTF_8));
|
||||
|
||||
assert (channel.writeInbound(eventBytes.buffer()));
|
||||
assertEquals(5, channel.inboundMessages().size());
|
||||
|
||||
RELPMessage event = channel.readInbound();
|
||||
assertEquals(RELPMessage.class, event.getClass());
|
||||
assertEquals(SYSLOG_FRAME_DATA, new String(event.getMessage(), StandardCharsets.UTF_8));
|
||||
assertEquals(2, channel.outboundMessages().size());
|
||||
}
|
||||
|
||||
private void sendFrames(final List<RELPFrame> frames, final OutputStream outputStream) throws IOException {
|
||||
RELPEncoder encoder = new RELPEncoder(StandardCharsets.UTF_8);
|
||||
for (final RELPFrame frame : frames) {
|
||||
final byte[] encodedFrame = encoder.encode(frame);
|
||||
outputStream.write(encodedFrame);
|
||||
outputStream.flush();
|
||||
}
|
||||
}
|
||||
|
||||
private List<RELPFrame> getFrames(final int syslogFrames) {
|
||||
final List<RELPFrame> frames = new ArrayList<>();
|
||||
frames.add(OPEN_FRAME);
|
||||
|
||||
for (int i = 0; i < syslogFrames; i++) {
|
||||
frames.add(SYSLOG_FRAME);
|
||||
}
|
||||
|
||||
frames.add(CLOSE_FRAME);
|
||||
return frames;
|
||||
}
|
||||
}
|
|
@ -1,45 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import io.netty.buffer.ByteBufOutputStream;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.embedded.EmbeddedChannel;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
class RELPResponseEncoderTest {
|
||||
|
||||
@Test
|
||||
void testEncodeRELPResponse() throws IOException {
|
||||
final byte[] relpResponse = new RELPChannelResponse(new RELPEncoder(Charset.defaultCharset()), RELPResponse.ok(321L)).toByteArray();
|
||||
|
||||
ByteBufOutputStream eventBytes = new ByteBufOutputStream(Unpooled.buffer(relpResponse.length));
|
||||
eventBytes.write(relpResponse);
|
||||
EmbeddedChannel channel = new EmbeddedChannel(new RELPResponseEncoder(Charset.defaultCharset()));
|
||||
|
||||
assert (channel.writeOutbound(eventBytes));
|
||||
ByteBufOutputStream result = channel.readOutbound();
|
||||
assertEquals("321 rsp 6 200 OK\n", new String(result.buffer().array()));
|
||||
}
|
||||
}
|
|
@ -1,141 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
public class TestRELPDecoder {
|
||||
|
||||
public static final String OPEN_FRAME_DATA = "relp_version=0\nrelp_software=librelp,1.2.7,http://librelp.adiscon.com\ncommands=syslog";
|
||||
public static final String OPEN_FRAME = "1 open 85 " + OPEN_FRAME_DATA + "\n";
|
||||
|
||||
public static final String SYSLOG_FRAME_DATA = "this is a syslog message here";
|
||||
public static final String SYSLOG_FRAME = "2 syslog 29 " + SYSLOG_FRAME_DATA + "\n";
|
||||
|
||||
public static final String CLOSE_FRAME = "3 close 0\n";
|
||||
|
||||
private RELPDecoder decoder;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() {
|
||||
this.decoder = new RELPDecoder(StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecodeSingleFrame() throws RELPFrameException {
|
||||
final byte[] input = OPEN_FRAME.getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
RELPFrame frame = null;
|
||||
for (byte b : input) {
|
||||
if (decoder.process(b)) {
|
||||
frame = decoder.getFrame();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
assertNotNull(frame);
|
||||
assertEquals(1, frame.getTxnr());
|
||||
assertEquals("open", frame.getCommand());
|
||||
assertEquals(85, frame.getDataLength());
|
||||
|
||||
assertNotNull(frame.getData());
|
||||
assertEquals(OPEN_FRAME_DATA, new String(frame.getData(), StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecodeMultipleCommands() throws RELPFrameException {
|
||||
final byte[] input = (OPEN_FRAME + SYSLOG_FRAME + CLOSE_FRAME).getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
List<RELPFrame> frames = new ArrayList<>();
|
||||
for (byte b : input) {
|
||||
if (decoder.process(b)) {
|
||||
frames.add(decoder.getFrame());
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(3, frames.size());
|
||||
|
||||
final RELPFrame frame1 = frames.get(0);
|
||||
assertNotNull(frame1);
|
||||
assertEquals(1, frame1.getTxnr());
|
||||
assertEquals("open", frame1.getCommand());
|
||||
assertEquals(85, frame1.getDataLength());
|
||||
|
||||
assertNotNull(frame1.getData());
|
||||
assertEquals(OPEN_FRAME_DATA, new String(frame1.getData(), StandardCharsets.UTF_8));
|
||||
|
||||
final RELPFrame frame2 = frames.get(1);
|
||||
assertNotNull(frame2);
|
||||
assertEquals(2, frame2.getTxnr());
|
||||
assertEquals("syslog", frame2.getCommand());
|
||||
assertEquals(29, frame2.getDataLength());
|
||||
|
||||
assertNotNull(frame2.getData());
|
||||
assertEquals(SYSLOG_FRAME_DATA, new String(frame2.getData(), StandardCharsets.UTF_8));
|
||||
|
||||
final RELPFrame frame3 = frames.get(2);
|
||||
assertNotNull(frame3);
|
||||
assertEquals(3, frame3.getTxnr());
|
||||
assertEquals("close", frame3.getCommand());
|
||||
assertEquals(0, frame3.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDecodeMultipleSyslogCommands() throws RELPFrameException {
|
||||
final String msg1 = "1 syslog 20 this is message 1234\n";
|
||||
final String msg2 = "2 syslog 22 this is message 456789\n";
|
||||
final String msg3 = "3 syslog 21 this is message ABCDE\n";
|
||||
final String msg = msg1 + msg2 + msg3;
|
||||
|
||||
final byte[] input = msg.getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
List<RELPFrame> frames = new ArrayList<>();
|
||||
|
||||
for (byte b : input) {
|
||||
if (decoder.process(b)) {
|
||||
frames.add(decoder.getFrame());
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(3, frames.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBadDataShouldThrowException() {
|
||||
assertThrows(RELPFrameException.class, () -> {
|
||||
final String msg = "NAN syslog 20 this is message 1234\n";
|
||||
final byte[] input = msg.getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
List<RELPFrame> frames = new ArrayList<>();
|
||||
|
||||
for (byte b : input) {
|
||||
if (decoder.process(b)) {
|
||||
frames.add(decoder.getFrame());
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
|
@ -1,90 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
|
||||
public class TestRELPEncoder {
|
||||
|
||||
@Test
|
||||
public void testEncodingWithData() {
|
||||
final RELPFrame frame = new RELPFrame.Builder()
|
||||
.txnr(1)
|
||||
.command("rsp")
|
||||
.dataLength(5)
|
||||
.data("12345".getBytes(StandardCharsets.UTF_8))
|
||||
.build();
|
||||
|
||||
final RELPEncoder encoder = new RELPEncoder(StandardCharsets.UTF_8);
|
||||
|
||||
final byte[] result = encoder.encode(frame);
|
||||
|
||||
final String expected = "1 rsp 5 12345\n";
|
||||
assertEquals(expected, new String(result, StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEncodingNoData() {
|
||||
final RELPFrame frame = new RELPFrame.Builder()
|
||||
.txnr(1)
|
||||
.command("rsp")
|
||||
.dataLength(0)
|
||||
.data(new byte[0])
|
||||
.build();
|
||||
|
||||
final RELPEncoder encoder = new RELPEncoder(StandardCharsets.UTF_8);
|
||||
|
||||
final byte[] result = encoder.encode(frame);
|
||||
|
||||
final String expected = "1 rsp 0\n";
|
||||
assertEquals(expected, new String(result, StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEncodingOpenResponse() {
|
||||
final String openFrameData = "relp_version=0\nrelp_software=librelp,1.2.7,http://librelp.adiscon.com\ncommands=syslog";
|
||||
final String openFrame = "1 open 85 " + openFrameData + "\n";
|
||||
System.out.println(openFrame);
|
||||
|
||||
final RELPDecoder decoder = new RELPDecoder(StandardCharsets.UTF_8);
|
||||
final RELPEncoder encoder = new RELPEncoder(StandardCharsets.UTF_8);
|
||||
|
||||
RELPFrame frame = null;
|
||||
for (byte b : openFrame.getBytes(StandardCharsets.UTF_8)) {
|
||||
if (decoder.process(b)) {
|
||||
frame = decoder.getFrame();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
assertNotNull(frame);
|
||||
|
||||
final Map<String, String> offers = RELPResponse.parseOffers(frame.getData(), StandardCharsets.UTF_8);
|
||||
final RELPFrame responseFrame = RELPResponse.open(frame.getTxnr(), offers).toFrame(StandardCharsets.UTF_8);
|
||||
|
||||
final byte[] response = encoder.encode(responseFrame);
|
||||
System.out.println(new String(response, StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
}
|
|
@ -1,59 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.frame;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
public class TestRELPFrame {
|
||||
|
||||
@Test
|
||||
public void testInvalidTxnr() {
|
||||
assertThrows(RELPFrameException.class, () -> {
|
||||
new RELPFrame.Builder().command("command").dataLength(5).data(new byte[5]).build();
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidCommand() {
|
||||
assertThrows(RELPFrameException.class, () -> {
|
||||
new RELPFrame.Builder().txnr(1).dataLength(5).data(new byte[5]).build();
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlankCommand() {
|
||||
assertThrows(RELPFrameException.class, () -> {
|
||||
new RELPFrame.Builder().txnr(1).command(" ").dataLength(5).data(new byte[5]).build();
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidDataLength() {
|
||||
assertThrows(RELPFrameException.class, () -> {
|
||||
new RELPFrame.Builder().txnr(1).command("command").data(new byte[5]).build();
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidData() {
|
||||
assertThrows(RELPFrameException.class, () -> {
|
||||
new RELPFrame.Builder().txnr(1).command("command").dataLength(5).data(null).build();
|
||||
});
|
||||
}
|
||||
}
|
|
@ -1,171 +0,0 @@
|
|||
/*
|
||||
* 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.standard.relp.response;
|
||||
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestRELPResponse {
|
||||
|
||||
@Test
|
||||
public void testResponseToFrame() {
|
||||
final long txnr = 123456789;
|
||||
final int code = RELPResponse.OK;
|
||||
final String message = "this is a message";
|
||||
final String data = "this is some data";
|
||||
|
||||
final RELPResponse response = new RELPResponse(txnr, code, message, data);
|
||||
|
||||
final RELPFrame frame = response.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + " " + message + "\n" + data;
|
||||
assertEquals(expected, result);
|
||||
assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResponseToFrameNoMessage() {
|
||||
final long txnr = 123456789;
|
||||
final int code = RELPResponse.OK;
|
||||
final String data = "this is some data";
|
||||
|
||||
final RELPResponse response = new RELPResponse(txnr, code, null, data);
|
||||
|
||||
final RELPFrame frame = response.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + "\n" + data;
|
||||
assertEquals(expected, result);
|
||||
assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResponseToFrameNoData() throws IOException {
|
||||
final long txnr = 123456789;
|
||||
final int code = RELPResponse.OK;
|
||||
final String message = "this is a message";
|
||||
|
||||
final RELPResponse response = new RELPResponse(txnr, code, message, null);
|
||||
|
||||
final RELPFrame frame = response.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + " " + message;
|
||||
assertEquals(expected, result);
|
||||
assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResponseToFrameNoDataNoMessage() throws IOException {
|
||||
final long txnr = 123456789;
|
||||
final int code = RELPResponse.OK;
|
||||
|
||||
final RELPResponse response = new RELPResponse(txnr, code);
|
||||
|
||||
final RELPFrame frame = response.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + "";
|
||||
assertEquals(expected, result);
|
||||
assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateOpenResponse() {
|
||||
final long txnr = 123456789;
|
||||
|
||||
final Map<String, String> offers = new HashMap<>();
|
||||
offers.put("key1", "val1");
|
||||
offers.put("key2", "val2");
|
||||
|
||||
final RELPResponse openResponse = RELPResponse.open(txnr, offers);
|
||||
|
||||
final RELPFrame frame = openResponse.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected1 = RELPResponse.OK + " OK\n" + "key1=val1\nkey2=val2";
|
||||
final String expected2 = RELPResponse.OK + " OK\n" + "key2=val2\nkey1=val1";
|
||||
assertTrue(result.equals(expected1) || result.equals(expected2));
|
||||
assertEquals(expected1.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateOpenResponseNoOffers() {
|
||||
final long txnr = 123456789;
|
||||
final Map<String, String> offers = new HashMap<>();
|
||||
|
||||
final RELPResponse openResponse = RELPResponse.open(txnr, offers);
|
||||
|
||||
final RELPFrame frame = openResponse.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = RELPResponse.OK + " OK\n";
|
||||
assertEquals(expected, result);
|
||||
assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateOkResponse() {
|
||||
final long txnr = 123456789;
|
||||
final RELPResponse openResponse = RELPResponse.ok(txnr);
|
||||
|
||||
final RELPFrame frame = openResponse.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = RELPResponse.OK + " OK";
|
||||
assertEquals(expected, result);
|
||||
assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateErrorResponse() {
|
||||
final long txnr = 123456789;
|
||||
final RELPResponse openResponse = RELPResponse.error(txnr);
|
||||
|
||||
final RELPFrame frame = openResponse.toFrame(StandardCharsets.UTF_8);
|
||||
assertEquals(txnr, frame.getTxnr());
|
||||
assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = RELPResponse.ERROR + " ERROR";
|
||||
assertEquals(expected, result);
|
||||
assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
}
|
|
@ -1,4 +0,0 @@
|
|||
{
|
||||
"id": 1,
|
||||
"name": "Mark",
|
||||
}
|
|
@ -1,5 +0,0 @@
|
|||
{
|
||||
"id": 1,
|
||||
"name": "Mark",
|
||||
"code": 48
|
||||
}
|
|
@ -1,5 +0,0 @@
|
|||
{
|
||||
"id": 1,
|
||||
"name": "Bool",
|
||||
"code": true
|
||||
}
|
|
@ -1,6 +0,0 @@
|
|||
{
|
||||
"id": 1,
|
||||
"name": "Mark",
|
||||
"code": 48,
|
||||
"extra": "another"
|
||||
}
|
|
@ -1,5 +0,0 @@
|
|||
{
|
||||
"id": 1,
|
||||
"name": "Mark",
|
||||
"code": null
|
||||
}
|
|
@ -1,4 +0,0 @@
|
|||
{
|
||||
"id": 1,
|
||||
"name": "Mark"
|
||||
}
|
|
@ -1,4 +0,0 @@
|
|||
{
|
||||
"name": "Mark",
|
||||
"code": 48
|
||||
}
|
|
@ -1,21 +0,0 @@
|
|||
[{
|
||||
"id": 1,
|
||||
"name": "Mark",
|
||||
"code": 48
|
||||
}, {
|
||||
"id": 2,
|
||||
"name": "George",
|
||||
"code": 48
|
||||
}, {
|
||||
"id": 3,
|
||||
"name": "Harry",
|
||||
"code": 21
|
||||
}, {
|
||||
"id": 4,
|
||||
"name": "Julie",
|
||||
"code": 48
|
||||
}, {
|
||||
"id": 82,
|
||||
"name": "Frank Henry",
|
||||
"code": 16
|
||||
}]
|
|
@ -22,9 +22,6 @@ import org.apache.nifi.components.Validator;
|
|||
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* This class is identical to {@code org.apache.nifi.processors.standard.ParseCEF.ValidateLocale}.
|
||||
*/
|
||||
public class ValidateLocale implements Validator {
|
||||
@Override
|
||||
public ValidationResult validate(String subject, String input, ValidationContext context) {
|
||||
|
|
|
@ -23,9 +23,7 @@
|
|||
<body>
|
||||
|
||||
<p>
|
||||
The CEFReader Controller Service serves as a mean to read and interpret CEF messages. Contrary to the ParseCEF Processor the
|
||||
CEFReader is intended to let the users access the whole range of record processing tools of NiFi, making it easy to work with a
|
||||
wide amount of incoming CEF messages.
|
||||
The CEFReader Controller Service serves as a mean to read and interpret CEF messages.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
|
|
Loading…
Reference in New Issue