mirror of https://github.com/apache/nifi.git
NIFI-1273 Adding ListenRELP processor which includes refactoring code that was previously part of ListenSyslog into a reusable framework for implementing listener processors, as well as back-end code for handling the RELP protocol. Addressing some feedback from review and fixing logging statements Addressing review feedback, added AsyncChannelDispatcher and made ChannelHandlerFactory use generics for the dispatcher This closes #179
Signed-off-by: Matt Gilman <matt.c.gilman@gmail.com>
This commit is contained in:
parent
2f5f7b8302
commit
1089f0a956
|
@ -36,5 +36,9 @@
|
|||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -0,0 +1,358 @@
|
|||
/*
|
||||
* 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.processor.util.listen;
|
||||
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
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.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.ChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
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;
|
||||
|
||||
/**
|
||||
* An abstract processor to extend from when listening for events over a channel. This processor
|
||||
* will start a ChannelDispatcher, and optionally a ChannelResponseDispatcher, in a background
|
||||
* thread which will end up placing events on a queue to polled by the onTrigger method. Sub-classes
|
||||
* are responsible for providing the dispatcher implementations.
|
||||
*
|
||||
* @param <E> the type of events being produced
|
||||
*/
|
||||
public abstract class AbstractListenEventProcessor<E extends Event> extends AbstractProcessor {
|
||||
|
||||
public static final PropertyDescriptor PORT = new PropertyDescriptor
|
||||
.Builder().name("Port")
|
||||
.description("The port to listen on for communication.")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.PORT_VALIDATOR)
|
||||
.build();
|
||||
public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
|
||||
.name("Character Set")
|
||||
.description("Specifies the character set of the received data.")
|
||||
.required(true)
|
||||
.defaultValue("UTF-8")
|
||||
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
|
||||
.build();
|
||||
public static final PropertyDescriptor RECV_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("Receive Buffer Size")
|
||||
.description("The size of each buffer used to receive messages. Adjust this value appropriately based on the expected size of the " +
|
||||
"incoming messages.")
|
||||
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||
.defaultValue("65507 B")
|
||||
.required(true)
|
||||
.build();
|
||||
public static final PropertyDescriptor MAX_SOCKET_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("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();
|
||||
|
||||
// Putting these properties here so sub-classes don't have to redefine them, but they are
|
||||
// not added to the properties by default since not all processors may need them
|
||||
|
||||
public static final PropertyDescriptor MAX_CONNECTIONS = new PropertyDescriptor.Builder()
|
||||
.name("Max Number of TCP Connections")
|
||||
.description("The maximum number of concurrent TCP connections to accept.")
|
||||
.addValidator(StandardValidators.createLongValidator(1, 65535, true))
|
||||
.defaultValue("2")
|
||||
.required(true)
|
||||
.build();
|
||||
public static final PropertyDescriptor MAX_BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||
.name("Max Batch Size")
|
||||
.description(
|
||||
"The maximum number of messages to add to a single FlowFile. If multiple messages are available, they will be concatenated along with "
|
||||
+ "the <Message Delimiter> up to this configured maximum number of messages")
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.expressionLanguageSupported(false)
|
||||
.defaultValue("1")
|
||||
.required(true)
|
||||
.build();
|
||||
public static final PropertyDescriptor MESSAGE_DELIMITER = new PropertyDescriptor.Builder()
|
||||
.name("Message Delimiter")
|
||||
.description("Specifies the delimiter to place between messages when multiple messages are bundled together (see <Max Batch Size> property).")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.defaultValue("\\n")
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("Messages received successfully will be sent out this relationship.")
|
||||
.build();
|
||||
|
||||
public static final int POLL_TIMEOUT_MS = 100;
|
||||
|
||||
private Set<Relationship> relationships;
|
||||
private List<PropertyDescriptor> descriptors;
|
||||
|
||||
protected volatile int port;
|
||||
protected volatile Charset charset;
|
||||
protected volatile ChannelDispatcher dispatcher;
|
||||
protected volatile BlockingQueue<E> events = new LinkedBlockingQueue<>(10);
|
||||
protected volatile BlockingQueue<E> errorEvents = new LinkedBlockingQueue<>();
|
||||
|
||||
@Override
|
||||
protected void init(final ProcessorInitializationContext context) {
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
descriptors.add(PORT);
|
||||
descriptors.add(RECV_BUFFER_SIZE);
|
||||
descriptors.add(MAX_SOCKET_BUFFER_SIZE);
|
||||
descriptors.add(CHARSET);
|
||||
descriptors.addAll(getAdditionalProperties());
|
||||
this.descriptors = Collections.unmodifiableList(descriptors);
|
||||
|
||||
final Set<Relationship> relationships = new HashSet<>();
|
||||
relationships.add(REL_SUCCESS);
|
||||
relationships.addAll(getAdditionalRelationships());
|
||||
this.relationships = Collections.unmodifiableSet(relationships);
|
||||
}
|
||||
|
||||
/**
|
||||
* Override to provide additional relationships for the processor.
|
||||
*
|
||||
* @return a list of relationships
|
||||
*/
|
||||
protected List<Relationship> getAdditionalRelationships() {
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
|
||||
/**
|
||||
* Override to provide additional properties for the processor.
|
||||
*
|
||||
* @return a list of properties
|
||||
*/
|
||||
protected List<PropertyDescriptor> getAdditionalProperties() {
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Set<Relationship> getRelationships() {
|
||||
return this.relationships;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
public void onScheduled(final ProcessContext context) throws IOException {
|
||||
charset = Charset.forName(context.getProperty(CHARSET).getValue());
|
||||
port = context.getProperty(PORT).asInteger();
|
||||
|
||||
final int maxChannelBufferSize = context.getProperty(MAX_SOCKET_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
|
||||
// create the dispatcher and call open() to bind to the given port
|
||||
dispatcher = createDispatcher(context, events);
|
||||
dispatcher.open(port, maxChannelBufferSize);
|
||||
|
||||
// start a thread to run the dispatcher
|
||||
final Thread readerThread = new Thread(dispatcher);
|
||||
readerThread.setName(getClass().getName() + " [" + getIdentifier() + "]");
|
||||
readerThread.setDaemon(true);
|
||||
readerThread.start();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param context the ProcessContext to retrieve property values from
|
||||
* @return a ChannelDispatcher to handle incoming connections
|
||||
*
|
||||
* @throws IOException if unable to listen on the requested port
|
||||
*/
|
||||
protected abstract ChannelDispatcher createDispatcher(final ProcessContext context, final BlockingQueue<E> events) throws IOException;
|
||||
|
||||
// used for testing to access the random port that was selected
|
||||
public final int getDispatcherPort() {
|
||||
return dispatcher == null ? 0 : dispatcher.getPort();
|
||||
}
|
||||
|
||||
public int getErrorQueueSize() {
|
||||
return errorEvents.size();
|
||||
}
|
||||
|
||||
@OnUnscheduled
|
||||
public void onUnscheduled() {
|
||||
if (dispatcher != null) {
|
||||
dispatcher.stop();
|
||||
dispatcher.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If pollErrorQueue is true, the error queue will be checked first and event will be
|
||||
* returned from the error queue if available.
|
||||
*
|
||||
* If pollErrorQueue is false, or no data is in the error queue, the regular queue is polled.
|
||||
*
|
||||
* If longPoll is true, the regular queue will be polled with a short timeout, otherwise it will
|
||||
* poll with no timeout which will return immediately.
|
||||
*
|
||||
* @param longPoll whether or not to poll the main queue with a small timeout
|
||||
* @param pollErrorQueue whether or not to poll the error queue first
|
||||
*
|
||||
* @return an event from one of the queues, or null if none are available
|
||||
*/
|
||||
protected E getMessage(final boolean longPoll, final boolean pollErrorQueue) {
|
||||
E event = null;
|
||||
if (pollErrorQueue) {
|
||||
event = errorEvents.poll();
|
||||
}
|
||||
|
||||
if (event == null) {
|
||||
try {
|
||||
if (longPoll) {
|
||||
event = events.poll(POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS);
|
||||
} else {
|
||||
event = events.poll();
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
return event;
|
||||
}
|
||||
|
||||
/**
|
||||
* Batches together up to the batchSize events. Events are grouped together based on a batch key which
|
||||
* by default is the sender of the event, but can be override by sub-classes.
|
||||
*
|
||||
* This method will return when batchSize has been reached, or when no more events are available on the queue.
|
||||
*
|
||||
* @param session the current session
|
||||
* @param totalBatchSize the total number of events to process
|
||||
* @param messageDemarcatorBytes the demarcator to put between messages when writing to a FlowFile
|
||||
*
|
||||
* @return a Map from the batch key to the FlowFile and events for that batch, the size of events in all
|
||||
* the batches will be <= batchSize
|
||||
*/
|
||||
protected Map<String,FlowFileEventBatch> getBatches(final ProcessSession session, final int totalBatchSize,
|
||||
final byte[] messageDemarcatorBytes) {
|
||||
|
||||
final Map<String,FlowFileEventBatch> batches = new HashMap<>();
|
||||
for (int i=0; i < totalBatchSize; i++) {
|
||||
final E event = getMessage(true, true);
|
||||
if (event == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
final String batchKey = getBatchKey(event);
|
||||
FlowFileEventBatch batch = batches.get(batchKey);
|
||||
|
||||
// if we don't have a batch for this key then create a new one
|
||||
if (batch == null) {
|
||||
batch = new FlowFileEventBatch(session.create(), new ArrayList<E>());
|
||||
batches.put(batchKey, batch);
|
||||
}
|
||||
|
||||
// add the current event to the batch
|
||||
batch.getEvents().add(event);
|
||||
|
||||
// append the event's data to the FlowFile, write the demarcator first if not on the first event
|
||||
final boolean writeDemarcator = (i > 0);
|
||||
try {
|
||||
final byte[] rawMessage = event.getData();
|
||||
FlowFile appendedFlowFile = session.append(batch.getFlowFile(), new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
if (writeDemarcator) {
|
||||
out.write(messageDemarcatorBytes);
|
||||
}
|
||||
|
||||
out.write(rawMessage);
|
||||
}
|
||||
});
|
||||
|
||||
// update the FlowFile reference in the batch object
|
||||
batch.setFlowFile(appendedFlowFile);
|
||||
|
||||
} catch (final Exception e) {
|
||||
getLogger().error("Failed to write contents of the message to FlowFile due to {}; will re-queue message and try again",
|
||||
new Object[] {e.getMessage()}, e);
|
||||
errorEvents.offer(event);
|
||||
break;
|
||||
}
|
||||
|
||||
session.adjustCounter("Messages Received", 1L, false);
|
||||
}
|
||||
|
||||
return batches;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param event an event that was pulled off the queue
|
||||
*
|
||||
* @return a key to use for batching events together, by default this uses the sender of the
|
||||
* event, but sub-classes should override this to batch by something else
|
||||
*/
|
||||
protected String getBatchKey(final E event) {
|
||||
return event.getSender();
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper to hold a FlowFile and the events that have been appended to it.
|
||||
*/
|
||||
protected final class FlowFileEventBatch {
|
||||
|
||||
private FlowFile flowFile;
|
||||
private List<E> events;
|
||||
|
||||
public FlowFileEventBatch(final FlowFile flowFile, final List<E> events) {
|
||||
this.flowFile = flowFile;
|
||||
this.events = events;
|
||||
}
|
||||
|
||||
public FlowFile getFlowFile() {
|
||||
return flowFile;
|
||||
}
|
||||
|
||||
public List<E> getEvents() {
|
||||
return events;
|
||||
}
|
||||
|
||||
public void setFlowFile(FlowFile flowFile) {
|
||||
this.flowFile = flowFile;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* 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.processor.util.listen.dispatcher;
|
||||
|
||||
import java.nio.channels.SelectionKey;
|
||||
|
||||
/**
|
||||
* A ChannelDispatcher that handles channels asynchronously.
|
||||
*/
|
||||
public interface AsyncChannelDispatcher extends ChannelDispatcher {
|
||||
|
||||
/**
|
||||
* Informs the dispatcher that the connection for the given key is complete.
|
||||
*
|
||||
* @param key a key that was previously selected
|
||||
*/
|
||||
void completeConnection(SelectionKey key);
|
||||
|
||||
/**
|
||||
* Informs the dispatcher that the connection for the given key can be added back for selection.
|
||||
*
|
||||
* @param key a key that was previously selected
|
||||
*/
|
||||
void addBackForSelection(SelectionKey key);
|
||||
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* 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.processor.util.listen.dispatcher;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Dispatches handlers for a given channel.
|
||||
*/
|
||||
public interface ChannelDispatcher extends Runnable {
|
||||
|
||||
/**
|
||||
* Opens the dispatcher listening on the given port and attempts to set the
|
||||
* OS socket buffer to maxBufferSize.
|
||||
*
|
||||
* @param port the port to listen on
|
||||
* @param maxBufferSize the size to set the OS socket buffer to
|
||||
* @throws IOException if an error occurred listening on the given port
|
||||
*/
|
||||
void open(int port, int maxBufferSize) throws IOException;
|
||||
|
||||
/**
|
||||
* @return the port being listened to
|
||||
*/
|
||||
int getPort();
|
||||
|
||||
/**
|
||||
* Stops the main dispatcher thread.
|
||||
*/
|
||||
void stop();
|
||||
|
||||
/**
|
||||
* Closes all listeners and stops all handler threads.
|
||||
*/
|
||||
void close();
|
||||
|
||||
}
|
|
@ -0,0 +1,155 @@
|
|||
/*
|
||||
* 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.processor.util.listen.dispatcher;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactoryUtil;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketAddress;
|
||||
import java.net.StandardSocketOptions;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.DatagramChannel;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.Selector;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Reads from the Datagram channel into an available buffer. If data is read then the buffer is queued for
|
||||
* processing, otherwise the buffer is returned to the buffer pool.
|
||||
*/
|
||||
public class DatagramChannelDispatcher<E extends Event<DatagramChannel>> implements ChannelDispatcher {
|
||||
|
||||
private final EventFactory<E> eventFactory;
|
||||
private final BlockingQueue<ByteBuffer> bufferPool;
|
||||
private final BlockingQueue<E> events;
|
||||
private final ProcessorLog logger;
|
||||
|
||||
private Selector selector;
|
||||
private DatagramChannel datagramChannel;
|
||||
private volatile boolean stopped = false;
|
||||
|
||||
public DatagramChannelDispatcher(final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<ByteBuffer> bufferPool,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
this.eventFactory = eventFactory;
|
||||
this.bufferPool = bufferPool;
|
||||
this.events = events;
|
||||
this.logger = logger;
|
||||
|
||||
if (bufferPool == null || bufferPool.size() == 0) {
|
||||
throw new IllegalArgumentException("A pool of available ByteBuffers is required");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(final int port, int maxBufferSize) throws IOException {
|
||||
datagramChannel = DatagramChannel.open();
|
||||
datagramChannel.configureBlocking(false);
|
||||
if (maxBufferSize > 0) {
|
||||
datagramChannel.setOption(StandardSocketOptions.SO_RCVBUF, maxBufferSize);
|
||||
final int actualReceiveBufSize = datagramChannel.getOption(StandardSocketOptions.SO_RCVBUF);
|
||||
if (actualReceiveBufSize < maxBufferSize) {
|
||||
logger.warn("Attempted to set Socket Buffer Size to " + maxBufferSize + " bytes but could only set to "
|
||||
+ actualReceiveBufSize + "bytes. You may want to consider changing the Operating System's "
|
||||
+ "maximum receive buffer");
|
||||
}
|
||||
}
|
||||
datagramChannel.socket().bind(new InetSocketAddress(port));
|
||||
selector = Selector.open();
|
||||
datagramChannel.register(selector, SelectionKey.OP_READ);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
final ByteBuffer buffer = bufferPool.poll();
|
||||
while (!stopped) {
|
||||
try {
|
||||
int selected = selector.select();
|
||||
if (selected > 0){
|
||||
Iterator<SelectionKey> selectorKeys = selector.selectedKeys().iterator();
|
||||
while (selectorKeys.hasNext()) {
|
||||
SelectionKey key = selectorKeys.next();
|
||||
selectorKeys.remove();
|
||||
if (!key.isValid()) {
|
||||
continue;
|
||||
}
|
||||
DatagramChannel channel = (DatagramChannel) key.channel();
|
||||
SocketAddress socketAddress;
|
||||
buffer.clear();
|
||||
while (!stopped && (socketAddress = channel.receive(buffer)) != null) {
|
||||
String sender = "";
|
||||
if (socketAddress instanceof InetSocketAddress) {
|
||||
sender = ((InetSocketAddress) socketAddress).getAddress().toString();
|
||||
}
|
||||
|
||||
// create a byte array from the buffer
|
||||
buffer.flip();
|
||||
byte bytes[] = new byte[buffer.limit()];
|
||||
buffer.get(bytes, 0, buffer.limit());
|
||||
|
||||
final Map<String,String> metadata = EventFactoryUtil.createMapWithSender(sender);
|
||||
final E event = eventFactory.create(bytes, metadata, null);
|
||||
|
||||
// queue the raw message with the sender, block until space is available
|
||||
events.put(event);
|
||||
buffer.clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
stopped = true;
|
||||
Thread.currentThread().interrupt();
|
||||
} catch (IOException e) {
|
||||
logger.error("Error reading from DatagramChannel", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (buffer != null) {
|
||||
try {
|
||||
bufferPool.put(buffer);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPort() {
|
||||
return datagramChannel == null ? 0 : datagramChannel.socket().getLocalPort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
selector.wakeup();
|
||||
stopped = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
IOUtils.closeQuietly(selector);
|
||||
IOUtils.closeQuietly(datagramChannel);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* 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.processor.util.listen.dispatcher;
|
||||
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
* Wrapper class so we can attach a buffer and/or an SSLSocketChannel to the selector key.
|
||||
* */
|
||||
public class SocketChannelAttachment {
|
||||
|
||||
private final ByteBuffer byteBuffer;
|
||||
private final SSLSocketChannel sslSocketChannel;
|
||||
|
||||
public SocketChannelAttachment(final ByteBuffer byteBuffer, final SSLSocketChannel sslSocketChannel) {
|
||||
this.byteBuffer = byteBuffer;
|
||||
this.sslSocketChannel = sslSocketChannel;
|
||||
}
|
||||
|
||||
public ByteBuffer getByteBuffer() {
|
||||
return byteBuffer;
|
||||
}
|
||||
|
||||
public SSLSocketChannel getSslSocketChannel() {
|
||||
return sslSocketChannel;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,243 @@
|
|||
/*
|
||||
* 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.processor.util.listen.dispatcher;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.StandardSocketOptions;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.Channel;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.Selector;
|
||||
import java.nio.channels.ServerSocketChannel;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Iterator;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* Accepts Socket connections on the given port and creates a handler for each connection to
|
||||
* be executed by a thread pool.
|
||||
*/
|
||||
public class SocketChannelDispatcher<E extends Event<SocketChannel>> implements AsyncChannelDispatcher {
|
||||
|
||||
private final EventFactory<E> eventFactory;
|
||||
private final ChannelHandlerFactory<E, AsyncChannelDispatcher> handlerFactory;
|
||||
private final BlockingQueue<ByteBuffer> bufferPool;
|
||||
private final BlockingQueue<E> events;
|
||||
private final ProcessorLog logger;
|
||||
private final int maxConnections;
|
||||
private final SSLContext sslContext;
|
||||
private final Charset charset;
|
||||
|
||||
private ExecutorService executor;
|
||||
private volatile boolean stopped = false;
|
||||
private Selector selector;
|
||||
private final BlockingQueue<SelectionKey> keyQueue;
|
||||
private final AtomicInteger currentConnections = new AtomicInteger(0);
|
||||
|
||||
|
||||
public SocketChannelDispatcher(final EventFactory<E> eventFactory,
|
||||
final ChannelHandlerFactory<E, AsyncChannelDispatcher> handlerFactory,
|
||||
final BlockingQueue<ByteBuffer> bufferPool,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger,
|
||||
final int maxConnections,
|
||||
final SSLContext sslContext,
|
||||
final Charset charset) {
|
||||
this.eventFactory = eventFactory;
|
||||
this.handlerFactory = handlerFactory;
|
||||
this.bufferPool = bufferPool;
|
||||
this.events = events;
|
||||
this.logger = logger;
|
||||
this.maxConnections = maxConnections;
|
||||
this.keyQueue = new LinkedBlockingQueue<>(maxConnections);
|
||||
this.sslContext = sslContext;
|
||||
this.charset = charset;
|
||||
|
||||
if (bufferPool == null || bufferPool.size() == 0 || bufferPool.size() != maxConnections) {
|
||||
throw new IllegalArgumentException(
|
||||
"A pool of available ByteBuffers equal to the maximum number of connections is required");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(final int port, int maxBufferSize) throws IOException {
|
||||
this.executor = Executors.newFixedThreadPool(maxConnections);
|
||||
|
||||
final ServerSocketChannel serverSocketChannel = ServerSocketChannel.open();
|
||||
serverSocketChannel.configureBlocking(false);
|
||||
if (maxBufferSize > 0) {
|
||||
serverSocketChannel.setOption(StandardSocketOptions.SO_RCVBUF, maxBufferSize);
|
||||
final int actualReceiveBufSize = serverSocketChannel.getOption(StandardSocketOptions.SO_RCVBUF);
|
||||
if (actualReceiveBufSize < maxBufferSize) {
|
||||
logger.warn("Attempted to set Socket Buffer Size to " + maxBufferSize + " bytes but could only set to "
|
||||
+ actualReceiveBufSize + "bytes. You may want to consider changing the Operating System's "
|
||||
+ "maximum receive buffer");
|
||||
}
|
||||
}
|
||||
serverSocketChannel.socket().bind(new InetSocketAddress(port));
|
||||
selector = Selector.open();
|
||||
serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (!stopped) {
|
||||
try {
|
||||
int selected = selector.select();
|
||||
if (selected > 0){
|
||||
Iterator<SelectionKey> selectorKeys = selector.selectedKeys().iterator();
|
||||
while (selectorKeys.hasNext()){
|
||||
SelectionKey key = selectorKeys.next();
|
||||
selectorKeys.remove();
|
||||
if (!key.isValid()){
|
||||
continue;
|
||||
}
|
||||
if (key.isAcceptable()) {
|
||||
// Handle new connections coming in
|
||||
final ServerSocketChannel channel = (ServerSocketChannel) key.channel();
|
||||
final SocketChannel socketChannel = channel.accept();
|
||||
// Check for available connections
|
||||
if (currentConnections.incrementAndGet() > maxConnections){
|
||||
currentConnections.decrementAndGet();
|
||||
logger.warn("Rejecting connection from {} because max connections has been met",
|
||||
new Object[]{ socketChannel.getRemoteAddress().toString() });
|
||||
IOUtils.closeQuietly(socketChannel);
|
||||
continue;
|
||||
}
|
||||
logger.debug("Accepted incoming connection from {}",
|
||||
new Object[]{socketChannel.getRemoteAddress().toString()});
|
||||
// Set socket to non-blocking, and register with selector
|
||||
socketChannel.configureBlocking(false);
|
||||
SelectionKey readKey = socketChannel.register(selector, SelectionKey.OP_READ);
|
||||
|
||||
// Prepare the byte buffer for the reads, clear it out
|
||||
ByteBuffer buffer = bufferPool.poll();
|
||||
buffer.clear();
|
||||
buffer.mark();
|
||||
|
||||
// If we have an SSLContext then create an SSLEngine for the channel
|
||||
SSLSocketChannel sslSocketChannel = null;
|
||||
if (sslContext != null) {
|
||||
final SSLEngine sslEngine = sslContext.createSSLEngine();
|
||||
sslSocketChannel = new SSLSocketChannel(sslEngine, socketChannel, false);
|
||||
}
|
||||
|
||||
// Attach the buffer and SSLSocketChannel to the key
|
||||
SocketChannelAttachment attachment = new SocketChannelAttachment(buffer, sslSocketChannel);
|
||||
readKey.attach(attachment);
|
||||
} else if (key.isReadable()) {
|
||||
// Clear out the operations the select is interested in until done reading
|
||||
key.interestOps(0);
|
||||
// Create a handler based on the protocol and whether an SSLEngine was provided or not
|
||||
final Runnable handler;
|
||||
if (sslContext != null) {
|
||||
handler = handlerFactory.createSSLHandler(key, this, charset, eventFactory, events, logger);
|
||||
} else {
|
||||
handler = handlerFactory.createHandler(key, this, charset, eventFactory, events, logger);
|
||||
}
|
||||
|
||||
// run the handler
|
||||
executor.execute(handler);
|
||||
}
|
||||
}
|
||||
}
|
||||
// Add back all idle sockets to the select
|
||||
SelectionKey key;
|
||||
while((key = keyQueue.poll()) != null){
|
||||
key.interestOps(SelectionKey.OP_READ);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.error("Error accepting connection from SocketChannel", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPort() {
|
||||
// Return the port for the key listening for accepts
|
||||
for(SelectionKey key : selector.keys()){
|
||||
if (key.isValid()) {
|
||||
final Channel channel = key.channel();
|
||||
if (channel instanceof ServerSocketChannel) {
|
||||
return ((ServerSocketChannel)channel).socket().getLocalPort();
|
||||
}
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
stopped = true;
|
||||
selector.wakeup();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// Wait a while for existing tasks to terminate
|
||||
if (!executor.awaitTermination(1000L, TimeUnit.MILLISECONDS)) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
} catch (InterruptedException ie) {
|
||||
// (Re-)Cancel if current thread also interrupted
|
||||
executor.shutdownNow();
|
||||
// Preserve interrupt status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
for(SelectionKey key : selector.keys()){
|
||||
IOUtils.closeQuietly(key.channel());
|
||||
}
|
||||
IOUtils.closeQuietly(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completeConnection(SelectionKey key) {
|
||||
// connection is done. Return the buffer to the pool
|
||||
SocketChannelAttachment attachment = (SocketChannelAttachment) key.attachment();
|
||||
try {
|
||||
bufferPool.put(attachment.getByteBuffer());
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
currentConnections.decrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBackForSelection(SelectionKey key) {
|
||||
keyQueue.offer(key);
|
||||
selector.wakeup();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processor.util.listen.event;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
|
||||
import java.nio.channels.SelectableChannel;
|
||||
|
||||
/**
|
||||
* An event that was read from a channel.
|
||||
*
|
||||
* @param <C> the type of SelectableChannel the event was read from
|
||||
*/
|
||||
public interface Event<C extends SelectableChannel> {
|
||||
|
||||
/**
|
||||
* @return the sending host of the data
|
||||
*/
|
||||
String getSender();
|
||||
|
||||
/**
|
||||
* @return raw data for this event
|
||||
*/
|
||||
byte[] getData();
|
||||
|
||||
/**
|
||||
* @return the responder to use for responding to this event, or null
|
||||
* if responses are not supported
|
||||
*/
|
||||
ChannelResponder<C> getResponder();
|
||||
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* 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.processor.util.listen.event;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Factory to create instances of a given type of Event.
|
||||
*/
|
||||
public interface EventFactory<E extends Event> {
|
||||
|
||||
/**
|
||||
* The key in the metadata map for the sender.
|
||||
*/
|
||||
String SENDER_KEY = "sender";
|
||||
|
||||
/**
|
||||
* Creates an event for the given data and metadata.
|
||||
*
|
||||
* @param data raw data from a channel
|
||||
* @param metadata additional metadata
|
||||
* @param responder a responder for the event with the channel populated
|
||||
*
|
||||
* @return an instance of the given type
|
||||
*/
|
||||
E create(final byte[] data, final Map<String, String> metadata, final ChannelResponder responder);
|
||||
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* 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.processor.util.listen.event;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Utility methods for EventFactory.
|
||||
*/
|
||||
public class EventFactoryUtil {
|
||||
|
||||
public static Map<String,String> createMapWithSender(final String sender) {
|
||||
Map<String,String> metadata = new HashMap<>();
|
||||
metadata.put(EventFactory.SENDER_KEY, sender);
|
||||
return metadata;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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.processor.util.listen.event;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
|
||||
import java.nio.channels.SelectableChannel;
|
||||
|
||||
/**
|
||||
* Standard implementation of Event.
|
||||
*/
|
||||
public class StandardEvent<C extends SelectableChannel> implements Event<C> {
|
||||
|
||||
private final String sender;
|
||||
private final byte[] data;
|
||||
private final ChannelResponder<C> responder;
|
||||
|
||||
public StandardEvent(final String sender, final byte[] data, final ChannelResponder<C> responder) {
|
||||
this.sender = sender;
|
||||
this.data = data;
|
||||
this.responder = responder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSender() {
|
||||
return sender;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getData() {
|
||||
return data;
|
||||
}
|
||||
|
||||
public ChannelResponder<C> getResponder() {
|
||||
return responder;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* 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.processor.util.listen.handler;
|
||||
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.ChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Base class for all channel handlers.
|
||||
*/
|
||||
public abstract class ChannelHandler<E extends Event, D extends ChannelDispatcher> implements Runnable {
|
||||
|
||||
protected final SelectionKey key;
|
||||
protected final D dispatcher;
|
||||
protected final Charset charset;
|
||||
protected final EventFactory<E> eventFactory;
|
||||
protected final BlockingQueue<E> events;
|
||||
protected final ProcessorLog logger;
|
||||
|
||||
|
||||
public ChannelHandler(final SelectionKey key,
|
||||
final D dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
this.key = key;
|
||||
this.dispatcher = dispatcher;
|
||||
this.charset = charset;
|
||||
this.eventFactory = eventFactory;
|
||||
this.events = events;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processor.util.listen.handler;
|
||||
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.ChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Factory that can produce ChannelHandlers for the given type of Event and ChannelDispatcher.
|
||||
*/
|
||||
public interface ChannelHandlerFactory<E extends Event, D extends ChannelDispatcher> {
|
||||
|
||||
ChannelHandler<E, D> createHandler(final SelectionKey key,
|
||||
final D dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger);
|
||||
|
||||
ChannelHandler<E, D> createSSLHandler(final SelectionKey key,
|
||||
final D dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger);
|
||||
}
|
|
@ -0,0 +1,152 @@
|
|||
/*
|
||||
* 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.processor.util.listen.handler.socket;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.SocketChannelAttachment;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactoryUtil;
|
||||
import org.apache.nifi.processor.util.listen.response.socket.SSLSocketChannelResponder;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
import org.apache.nifi.stream.io.ByteArrayOutputStream;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Wraps a SocketChannel with an SSLSocketChannel for receiving messages over TLS.
|
||||
*/
|
||||
public class SSLSocketChannelHandler<E extends Event<SocketChannel>> extends SocketChannelHandler<E> {
|
||||
|
||||
private final ByteArrayOutputStream currBytes = new ByteArrayOutputStream(4096);
|
||||
|
||||
public SSLSocketChannelHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
super(key, dispatcher, charset, eventFactory, events, logger);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
boolean eof = false;
|
||||
SSLSocketChannel sslSocketChannel = null;
|
||||
try {
|
||||
int bytesRead;
|
||||
final SocketChannel socketChannel = (SocketChannel) key.channel();
|
||||
final SocketChannelAttachment attachment = (SocketChannelAttachment) key.attachment();
|
||||
|
||||
// get the SSLSocketChannel from the attachment
|
||||
sslSocketChannel = attachment.getSslSocketChannel();
|
||||
|
||||
// SSLSocketChannel deals with byte[] so ByteBuffer isn't used here, but we'll use the size to create a new byte[]
|
||||
final ByteBuffer socketBuffer = attachment.getByteBuffer();
|
||||
byte[] socketBufferArray = new byte[socketBuffer.limit()];
|
||||
|
||||
// read until no more data
|
||||
try {
|
||||
while ((bytesRead = sslSocketChannel.read(socketBufferArray)) > 0) {
|
||||
processBuffer(sslSocketChannel, socketChannel, bytesRead, socketBufferArray);
|
||||
logger.debug("bytes read from sslSocketChannel {}", new Object[]{bytesRead});
|
||||
}
|
||||
} catch (SocketTimeoutException ste) {
|
||||
// SSLSocketChannel will throw this exception when 0 bytes are read and the timeout threshold
|
||||
// is exceeded, we don't want to close the connection in this case
|
||||
bytesRead = 0;
|
||||
}
|
||||
|
||||
// Check for closed socket
|
||||
if( bytesRead < 0 ){
|
||||
eof = true;
|
||||
logger.debug("Reached EOF, closing connection");
|
||||
} else {
|
||||
logger.debug("No more data available, returning for selection");
|
||||
}
|
||||
} catch (ClosedByInterruptException | InterruptedException e) {
|
||||
logger.debug("read loop interrupted, closing connection");
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} catch (ClosedChannelException e) {
|
||||
// ClosedChannelException doesn't have a message so handle it separately from IOException
|
||||
logger.error("Error reading from channel due to channel being closed", e);
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} catch (IOException e) {
|
||||
logger.error("Error reading from channel due to {}", new Object[] {e.getMessage()}, e);
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} finally {
|
||||
if(eof == true) {
|
||||
IOUtils.closeQuietly(sslSocketChannel);
|
||||
dispatcher.completeConnection(key);
|
||||
} else {
|
||||
dispatcher.addBackForSelection(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the contents of the buffer. Give sub-classes a chance to override this behavior.
|
||||
*
|
||||
* @param sslSocketChannel the channel the data was read from
|
||||
* @param socketChannel the socket channel being wrapped by sslSocketChannel
|
||||
* @param bytesRead the number of bytes read
|
||||
* @param buffer the buffer to process
|
||||
* @throws InterruptedException thrown if interrupted while queuing events
|
||||
*/
|
||||
protected void processBuffer(final SSLSocketChannel sslSocketChannel, final SocketChannel socketChannel,
|
||||
final int bytesRead, final byte[] buffer) throws InterruptedException, IOException {
|
||||
final InetAddress sender = socketChannel.socket().getInetAddress();
|
||||
|
||||
// go through the buffer looking for the end of each message
|
||||
for (int i = 0; i < bytesRead; i++) {
|
||||
final byte currByte = buffer[i];
|
||||
currBytes.write(currByte);
|
||||
|
||||
// check if at end of a message
|
||||
if (currByte == getDelimiter()) {
|
||||
final SSLSocketChannelResponder response = new SSLSocketChannelResponder(socketChannel, sslSocketChannel);
|
||||
final Map<String,String> metadata = EventFactoryUtil.createMapWithSender(sender.toString());
|
||||
|
||||
// queue the raw event blocking until space is available, reset the temporary buffer
|
||||
final E event = eventFactory.create(currBytes.toByteArray(), metadata, response);
|
||||
events.put(event);
|
||||
currBytes.reset();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getDelimiter() {
|
||||
return TCP_DELIMITER;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* 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.processor.util.listen.handler.socket;
|
||||
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandler;
|
||||
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Base class for socket channel handlers.
|
||||
*/
|
||||
public abstract class SocketChannelHandler<E extends Event<SocketChannel>> extends ChannelHandler<E, AsyncChannelDispatcher> {
|
||||
|
||||
static final byte TCP_DELIMITER = '\n';
|
||||
|
||||
public SocketChannelHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
super(key, dispatcher, charset, eventFactory, events, logger);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the byte used as the delimiter between messages for the given handler
|
||||
*/
|
||||
public abstract byte getDelimiter();
|
||||
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* 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.processor.util.listen.handler.socket;
|
||||
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandler;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory;
|
||||
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Default factory for creating socket channel handlers.
|
||||
*/
|
||||
public class SocketChannelHandlerFactory<E extends Event<SocketChannel>> implements ChannelHandlerFactory<E, AsyncChannelDispatcher> {
|
||||
|
||||
@Override
|
||||
public ChannelHandler<E, AsyncChannelDispatcher> createHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
return new StandardSocketChannelHandler<>(key, dispatcher, charset, eventFactory, events, logger);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelHandler<E, AsyncChannelDispatcher> createSSLHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
return new SSLSocketChannelHandler<>(key, dispatcher, charset, eventFactory, events, logger);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,157 @@
|
|||
/*
|
||||
* 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.processor.util.listen.handler.socket;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.SocketChannelAttachment;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactoryUtil;
|
||||
import org.apache.nifi.processor.util.listen.response.socket.SocketChannelResponder;
|
||||
import org.apache.nifi.stream.io.ByteArrayOutputStream;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Reads from the given SocketChannel into the provided buffer. If the given delimiter is found, the data
|
||||
* read up to that point is queued for processing.
|
||||
*/
|
||||
public class StandardSocketChannelHandler<E extends Event<SocketChannel>> extends SocketChannelHandler<E> {
|
||||
|
||||
private final ByteArrayOutputStream currBytes = new ByteArrayOutputStream(4096);
|
||||
|
||||
public StandardSocketChannelHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
super(key, dispatcher, charset, eventFactory, events, logger);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
boolean eof = false;
|
||||
SocketChannel socketChannel = null;
|
||||
|
||||
try {
|
||||
int bytesRead;
|
||||
socketChannel = (SocketChannel) key.channel();
|
||||
|
||||
final SocketChannelAttachment attachment = (SocketChannelAttachment) key.attachment();
|
||||
final ByteBuffer socketBuffer = attachment.getByteBuffer();
|
||||
|
||||
// read until the buffer is full
|
||||
while ((bytesRead = socketChannel.read(socketBuffer)) > 0) {
|
||||
// prepare byte buffer for reading
|
||||
socketBuffer.flip();
|
||||
// mark the current position as start, in case of partial message read
|
||||
socketBuffer.mark();
|
||||
// process the contents that have been read into the buffer
|
||||
processBuffer(socketChannel, socketBuffer);
|
||||
|
||||
// Preserve bytes in buffer for next call to run
|
||||
// NOTE: This code could benefit from the two ByteBuffer read calls to avoid
|
||||
// this compact for higher throughput
|
||||
socketBuffer.reset();
|
||||
socketBuffer.compact();
|
||||
logger.debug("bytes read {}", new Object[]{bytesRead});
|
||||
}
|
||||
|
||||
// Check for closed socket
|
||||
if( bytesRead < 0 ){
|
||||
eof = true;
|
||||
logger.debug("Reached EOF, closing connection");
|
||||
} else {
|
||||
logger.debug("No more data available, returning for selection");
|
||||
}
|
||||
} catch (ClosedByInterruptException | InterruptedException e) {
|
||||
logger.debug("read loop interrupted, closing connection");
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} catch (ClosedChannelException e) {
|
||||
// ClosedChannelException doesn't have a message so handle it separately from IOException
|
||||
logger.error("Error reading from channel due to channel being closed", e);
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} catch (IOException e) {
|
||||
logger.error("Error reading from channel due to {}", new Object[] {e.getMessage()}, e);
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} finally {
|
||||
if(eof == true) {
|
||||
IOUtils.closeQuietly(socketChannel);
|
||||
dispatcher.completeConnection(key);
|
||||
} else {
|
||||
dispatcher.addBackForSelection(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the contents that have been read into the buffer. Allow sub-classes to override this behavior.
|
||||
*
|
||||
* @param socketChannel the channel the data was read from
|
||||
* @param socketBuffer the buffer the data was read into
|
||||
* @throws InterruptedException if interrupted when queuing events
|
||||
*/
|
||||
protected void processBuffer(final SocketChannel socketChannel, final ByteBuffer socketBuffer) throws InterruptedException, IOException {
|
||||
// get total bytes in buffer
|
||||
final int total = socketBuffer.remaining();
|
||||
final InetAddress sender = socketChannel.socket().getInetAddress();
|
||||
|
||||
// go through the buffer looking for the end of each message
|
||||
currBytes.reset();
|
||||
for (int i = 0; i < total; i++) {
|
||||
// NOTE: For higher throughput, the looking for \n and copying into the byte stream could be improved
|
||||
// Pull data out of buffer and cram into byte array
|
||||
byte currByte = socketBuffer.get();
|
||||
currBytes.write(currByte);
|
||||
|
||||
// check if at end of a message
|
||||
if (currByte == getDelimiter()) {
|
||||
final SocketChannelResponder response = new SocketChannelResponder(socketChannel);
|
||||
final Map<String,String> metadata = EventFactoryUtil.createMapWithSender(sender.toString());
|
||||
|
||||
// queue the raw event blocking until space is available, reset the buffer
|
||||
final E event = eventFactory.create(currBytes.toByteArray(), metadata, response);
|
||||
events.put(event);
|
||||
currBytes.reset();
|
||||
|
||||
// Mark this as the start of the next message
|
||||
socketBuffer.mark();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getDelimiter() {
|
||||
return TCP_DELIMITER;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* 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.processor.util.listen.response;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.SelectableChannel;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A responder for a given channel.
|
||||
*
|
||||
* @param <C> The type of SelectableChannel where the response will be written.
|
||||
*/
|
||||
public interface ChannelResponder<C extends SelectableChannel> {
|
||||
|
||||
/**
|
||||
* @return a SelectableChannel to write the response to
|
||||
*/
|
||||
C getChannel();
|
||||
|
||||
/**
|
||||
* @return a list of responses to write to the channel
|
||||
*/
|
||||
List<ChannelResponse> getResponses();
|
||||
|
||||
/**
|
||||
* @param response adds the given response to the list of responses
|
||||
*/
|
||||
void addResponse(ChannelResponse response);
|
||||
|
||||
/**
|
||||
* Writes the responses to the underlying channel.
|
||||
*/
|
||||
void respond() throws IOException;
|
||||
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* 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.processor.util.listen.response;
|
||||
|
||||
/**
|
||||
* A response to send back over channel.
|
||||
*/
|
||||
public interface ChannelResponse {
|
||||
|
||||
/**
|
||||
* @return the bytes that should be written to a channel for this response
|
||||
*/
|
||||
byte[] toByteArray();
|
||||
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* 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.processor.util.listen.response.socket;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponse;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.SocketChannel;
|
||||
|
||||
/**
|
||||
* A ChannelResponder for SSLSocketChannels.
|
||||
*/
|
||||
public class SSLSocketChannelResponder extends SocketChannelResponder {
|
||||
|
||||
private SSLSocketChannel sslSocketChannel;
|
||||
|
||||
public SSLSocketChannelResponder(final SocketChannel socketChannel, final SSLSocketChannel sslSocketChannel) {
|
||||
super(socketChannel);
|
||||
this.sslSocketChannel = sslSocketChannel;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void respond() throws IOException {
|
||||
for (final ChannelResponse response : responses) {
|
||||
sslSocketChannel.write(response.toByteArray());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
/*
|
||||
* 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.processor.util.listen.response.socket;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponse;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A ChannelResponder for SocketChannels. The SocketChannel should first be registered with a selector,
|
||||
* upon being selected for writing the respond() method should be executed.
|
||||
*/
|
||||
public class SocketChannelResponder implements ChannelResponder<SocketChannel> {
|
||||
|
||||
protected final List<ChannelResponse> responses;
|
||||
protected final SocketChannel socketChannel;
|
||||
|
||||
public SocketChannelResponder(final SocketChannel socketChannel) {
|
||||
this.responses = new ArrayList<>();
|
||||
this.socketChannel = socketChannel;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SocketChannel getChannel() {
|
||||
return socketChannel;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ChannelResponse> getResponses() {
|
||||
return Collections.unmodifiableList(responses);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addResponse(ChannelResponse response) {
|
||||
this.responses.add(response);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void respond() throws IOException {
|
||||
for (final ChannelResponse response : responses) {
|
||||
final ByteBuffer responseBuffer = ByteBuffer.wrap(response.toByteArray());
|
||||
|
||||
while (responseBuffer.hasRemaining()) {
|
||||
socketChannel.write(responseBuffer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -18,7 +18,6 @@ package org.apache.nifi.processors.standard;
|
|||
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.attributes.FlowFileAttributeKey;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
|
@ -32,7 +31,7 @@ public abstract class AbstractSyslogProcessor extends AbstractProcessor {
|
|||
|
||||
public static final PropertyDescriptor PROTOCOL = new PropertyDescriptor
|
||||
.Builder().name("Protocol")
|
||||
.description("The protocol for Syslog communication, either TCP or UDP.")
|
||||
.description("The protocol for Syslog communication.")
|
||||
.required(true)
|
||||
.allowableValues(TCP_VALUE, UDP_VALUE)
|
||||
.defaultValue(UDP_VALUE.getValue())
|
||||
|
@ -45,39 +44,11 @@ public abstract class AbstractSyslogProcessor extends AbstractProcessor {
|
|||
.build();
|
||||
public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
|
||||
.name("Character Set")
|
||||
.description("Specifies which character set of the Syslog messages")
|
||||
.description("Specifies the character set of the Syslog messages")
|
||||
.required(true)
|
||||
.defaultValue("UTF-8")
|
||||
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
|
||||
.build();
|
||||
|
||||
|
||||
/**
|
||||
* FlowFile Attributes for each Syslog message.
|
||||
*/
|
||||
public enum SyslogAttributes implements FlowFileAttributeKey {
|
||||
PRIORITY("syslog.priority"),
|
||||
SEVERITY("syslog.severity"),
|
||||
FACILITY("syslog.facility"),
|
||||
VERSION("syslog.version"),
|
||||
TIMESTAMP("syslog.timestamp"),
|
||||
HOSTNAME("syslog.hostname"),
|
||||
SENDER("syslog.sender"),
|
||||
BODY("syslog.body"),
|
||||
VALID("syslog.valid"),
|
||||
PROTOCOL("syslog.protocol"),
|
||||
PORT("syslog.port");
|
||||
|
||||
private String key;
|
||||
|
||||
SyslogAttributes(String key) {
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String key() {
|
||||
return key;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -125,7 +125,7 @@ public class HandleHttpResponse extends AbstractProcessor {
|
|||
final String statusCodeValue = context.getProperty(STATUS_CODE).evaluateAttributeExpressions(flowFile).getValue();
|
||||
if (!isNumber(statusCodeValue)) {
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
getLogger().error("Failed to response to HTTP request for {} because status code was '{}', which is not a valid number", new Object[]{flowFile, statusCodeValue});
|
||||
getLogger().error("Failed to respond to HTTP request for {} because status code was '{}', which is not a valid number", new Object[]{flowFile, statusCodeValue});
|
||||
}
|
||||
|
||||
final HttpContextMap contextMap = context.getProperty(HTTP_CONTEXT_MAP).asControllerService(HttpContextMap.class);
|
||||
|
|
|
@ -0,0 +1,225 @@
|
|||
/*
|
||||
* 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.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.flowfile.attributes.FlowFileAttributeKey;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.listen.AbstractListenEventProcessor;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.ChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.SocketChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPEvent;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPEventFactory;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPEncoder;
|
||||
import org.apache.nifi.processors.standard.relp.handler.RELPSocketChannelHandlerFactory;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
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 AbstractListenEventProcessor<RELPEvent> {
|
||||
|
||||
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("SSL Context Service")
|
||||
.description("The Controller Service to use in order to obtain an SSL Context. If this property is set, " +
|
||||
"messages will be received over a secure connection.")
|
||||
.required(false)
|
||||
.identifiesControllerService(SSLContextService.class)
|
||||
.build();
|
||||
|
||||
private volatile RELPEncoder relpEncoder;
|
||||
private volatile byte[] messageDemarcatorBytes; //it is only the array reference that is volatile - not the contents.
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getAdditionalProperties() {
|
||||
return Arrays.asList(MAX_CONNECTIONS, MAX_BATCH_SIZE, MESSAGE_DELIMITER, SSL_CONTEXT_SERVICE);
|
||||
}
|
||||
|
||||
@Override
|
||||
@OnScheduled
|
||||
public void onScheduled(ProcessContext context) throws IOException {
|
||||
super.onScheduled(context);
|
||||
// wanted to ensure charset was already populated here
|
||||
relpEncoder = new RELPEncoder(charset);
|
||||
|
||||
final String msgDemarcator = context.getProperty(MESSAGE_DELIMITER).getValue().replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
||||
messageDemarcatorBytes = msgDemarcator.getBytes(charset);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ChannelDispatcher createDispatcher(final ProcessContext context, final BlockingQueue<RELPEvent> events) throws IOException {
|
||||
final EventFactory<RELPEvent> eventFactory = new RELPEventFactory();
|
||||
final ChannelHandlerFactory<RELPEvent,AsyncChannelDispatcher> handlerFactory = new RELPSocketChannelHandlerFactory<>();
|
||||
|
||||
final int maxConnections = context.getProperty(MAX_CONNECTIONS).asInteger();
|
||||
final int bufferSize = context.getProperty(RECV_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
final Charset charSet = Charset.forName(context.getProperty(CHARSET).getValue());
|
||||
|
||||
// initialize the buffer pool based on max number of connections and the buffer size
|
||||
final LinkedBlockingQueue<ByteBuffer> bufferPool = new LinkedBlockingQueue<>(maxConnections);
|
||||
for (int i = 0; i < maxConnections; i++) {
|
||||
bufferPool.offer(ByteBuffer.allocate(bufferSize));
|
||||
}
|
||||
|
||||
// if an SSLContextService was provided then create an SSLContext to pass down to the dispatcher
|
||||
SSLContext sslContext = null;
|
||||
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
||||
if (sslContextService != null) {
|
||||
sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED);
|
||||
}
|
||||
|
||||
// if we decide to support SSL then get the context and pass it in here
|
||||
return new SocketChannelDispatcher<>(eventFactory, handlerFactory, bufferPool, events,
|
||||
getLogger(), maxConnections, sslContext, charSet);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
final int maxBatchSize = context.getProperty(MAX_BATCH_SIZE).asInteger();
|
||||
final Map<String,FlowFileEventBatch> batches = getBatches(session, maxBatchSize, messageDemarcatorBytes);
|
||||
|
||||
// if the size is 0 then there was nothing to process so yield and return
|
||||
if (batches.size() == 0) {
|
||||
context.yield();
|
||||
return;
|
||||
}
|
||||
|
||||
for (Map.Entry<String,FlowFileEventBatch> entry : batches.entrySet()) {
|
||||
FlowFile flowFile = entry.getValue().getFlowFile();
|
||||
final List<RELPEvent> 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", new Object[] {entry.getKey()});
|
||||
continue;
|
||||
}
|
||||
|
||||
// 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()));
|
||||
}
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
||||
getLogger().debug("Transferring {} to success", new Object[] {flowFile});
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
|
||||
// create a provenance receive event
|
||||
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();
|
||||
session.getProvenanceReporter().receive(flowFile, transitUri);
|
||||
|
||||
// commit the session to guarantee the data has been delivered
|
||||
session.commit();
|
||||
|
||||
// respond to each event to acknowledge successful receipt
|
||||
for (final RELPEvent event : events) {
|
||||
respond(event, RELPResponse.ok(event.getTxnr()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getBatchKey(RELPEvent event) {
|
||||
return event.getSender() + "_" + event.getCommand();
|
||||
}
|
||||
|
||||
protected void respond(final RELPEvent event, final RELPResponse relpResponse) {
|
||||
final ChannelResponse response = new RELPChannelResponse(relpEncoder, relpResponse);
|
||||
|
||||
final ChannelResponder responder = event.getResponder();
|
||||
responder.addResponse(response);
|
||||
try {
|
||||
responder.respond();
|
||||
} catch (IOException e) {
|
||||
getLogger().error("Error sending response for transaction {} due to {}",
|
||||
new Object[] {event.getTxnr(), e.getMessage()}, e);
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -31,7 +30,6 @@ import org.apache.nifi.components.ValidationContext;
|
|||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
|
@ -40,26 +38,25 @@ import org.apache.nifi.processor.Relationship;
|
|||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.SyslogEvent;
|
||||
import org.apache.nifi.processors.standard.util.SyslogParser;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.ChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.DatagramChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.SocketChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.socket.SocketChannelHandlerFactory;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogAttributes;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogEvent;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogParser;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
import org.apache.nifi.stream.io.ByteArrayOutputStream;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketAddress;
|
||||
import java.net.StandardSocketOptions;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.Channel;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.channels.DatagramChannel;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.Selector;
|
||||
import java.nio.channels.ServerSocketChannel;
|
||||
import java.nio.channels.SelectableChannel;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
|
@ -67,16 +64,12 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
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.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
|
||||
|
@ -175,7 +168,7 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
private Set<Relationship> relationships;
|
||||
private List<PropertyDescriptor> descriptors;
|
||||
|
||||
private volatile ChannelReader channelReader;
|
||||
private volatile ChannelDispatcher channelDispatcher;
|
||||
private volatile SyslogParser parser;
|
||||
private volatile BlockingQueue<ByteBuffer> bufferPool;
|
||||
private volatile BlockingQueue<RawSyslogEvent> syslogEvents = new LinkedBlockingQueue<>(10);
|
||||
|
@ -255,11 +248,10 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
final String protocol = context.getProperty(PROTOCOL).getValue();
|
||||
final String charSet = context.getProperty(CHARSET).getValue();
|
||||
final String msgDemarcator = context.getProperty(MESSAGE_DELIMITER).getValue().replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
||||
final String charsetName = context.getProperty(CHARSET).getValue();
|
||||
messageDemarcatorBytes = msgDemarcator.getBytes(Charset.forName(charsetName));
|
||||
messageDemarcatorBytes = msgDemarcator.getBytes(Charset.forName(charSet));
|
||||
|
||||
final int maxConnections;
|
||||
if (protocol.equals(UDP_VALUE.getValue())) {
|
||||
if (UDP_VALUE.getValue().equals(protocol)) {
|
||||
maxConnections = 1;
|
||||
} else {
|
||||
maxConnections = context.getProperty(MAX_CONNECTIONS).asLong().intValue();
|
||||
|
@ -274,10 +266,10 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
|
||||
// create either a UDP or TCP reader and call open() to bind to the given port
|
||||
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
||||
channelReader = createChannelReader(protocol, bufferPool, syslogEvents, maxConnections, sslContextService);
|
||||
channelReader.open(port, maxChannelBufferSize);
|
||||
channelDispatcher = createChannelReader(protocol, bufferPool, syslogEvents, maxConnections, sslContextService, Charset.forName(charSet));
|
||||
channelDispatcher.open(port, maxChannelBufferSize);
|
||||
|
||||
final Thread readerThread = new Thread(channelReader);
|
||||
final Thread readerThread = new Thread(channelDispatcher);
|
||||
readerThread.setName("ListenSyslog [" + getIdentifier() + "]");
|
||||
readerThread.setDaemon(true);
|
||||
readerThread.start();
|
||||
|
@ -288,12 +280,15 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
return parser;
|
||||
}
|
||||
|
||||
// visible for testing to be overridden and provide a mock ChannelReader if desired
|
||||
protected ChannelReader createChannelReader(final String protocol, final BlockingQueue<ByteBuffer> bufferPool, final BlockingQueue<RawSyslogEvent> syslogEvents,
|
||||
int maxConnections, final SSLContextService sslContextService)
|
||||
throws IOException {
|
||||
if (protocol.equals(UDP_VALUE.getValue())) {
|
||||
return new DatagramChannelReader(bufferPool, syslogEvents, getLogger());
|
||||
// visible for testing to be overridden and provide a mock ChannelDispatcher if desired
|
||||
protected ChannelDispatcher createChannelReader(final String protocol, final BlockingQueue<ByteBuffer> bufferPool,
|
||||
final BlockingQueue<RawSyslogEvent> events, final int maxConnections,
|
||||
final SSLContextService sslContextService, final Charset charset) throws IOException {
|
||||
|
||||
final EventFactory<RawSyslogEvent> eventFactory = new RawSyslogEventFactory();
|
||||
|
||||
if (UDP_VALUE.getValue().equals(protocol)) {
|
||||
return new DatagramChannelDispatcher(eventFactory, bufferPool, events, getLogger());
|
||||
} else {
|
||||
// if an SSLContextService was provided then create an SSLContext to pass down to the dispatcher
|
||||
SSLContext sslContext = null;
|
||||
|
@ -301,20 +296,21 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED);
|
||||
}
|
||||
|
||||
return new SocketChannelDispatcher(bufferPool, syslogEvents, getLogger(), maxConnections, sslContext);
|
||||
final ChannelHandlerFactory<RawSyslogEvent<SocketChannel>, AsyncChannelDispatcher> handlerFactory = new SocketChannelHandlerFactory<>();
|
||||
return new SocketChannelDispatcher(eventFactory, handlerFactory, bufferPool, events, getLogger(), maxConnections, sslContext, charset);
|
||||
}
|
||||
}
|
||||
|
||||
// used for testing to access the random port that was selected
|
||||
protected int getPort() {
|
||||
return channelReader == null ? 0 : channelReader.getPort();
|
||||
return channelDispatcher == null ? 0 : channelDispatcher.getPort();
|
||||
}
|
||||
|
||||
@OnUnscheduled
|
||||
public void onUnscheduled() {
|
||||
if (channelReader != null) {
|
||||
channelReader.stop();
|
||||
channelReader.close();
|
||||
if (channelDispatcher != null) {
|
||||
channelDispatcher.stop();
|
||||
channelDispatcher.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -394,7 +390,7 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
if (shouldParse) {
|
||||
boolean valid = true;
|
||||
try {
|
||||
event = parser.parseEvent(rawSyslogEvent.getRawMessage(), sender);
|
||||
event = parser.parseEvent(rawSyslogEvent.getData(), sender);
|
||||
} catch (final ProcessException pe) {
|
||||
getLogger().warn("Failed to parse Syslog event; routing to invalid");
|
||||
valid = false;
|
||||
|
@ -411,7 +407,7 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
}
|
||||
|
||||
try {
|
||||
final byte[] rawBytes = rawSyslogEvent.getRawMessage();
|
||||
final byte[] rawBytes = rawSyslogEvent.getData();
|
||||
invalidFlowFile = session.write(invalidFlowFile, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
|
@ -449,7 +445,7 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
|
||||
try {
|
||||
// write the raw bytes of the message as the FlowFile content
|
||||
final byte[] rawMessage = (event == null) ? rawSyslogEvent.getRawMessage() : event.getRawMessage();
|
||||
final byte[] rawMessage = (event == null) ? rawSyslogEvent.getData() : event.getRawMessage();
|
||||
flowFile = session.append(flowFile, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
|
@ -495,505 +491,47 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
}
|
||||
|
||||
/**
|
||||
* Reads messages from a channel until told to stop.
|
||||
* Wrapper class to pass around the raw message and the host/ip that sent it
|
||||
*/
|
||||
private interface ChannelReader extends Runnable {
|
||||
|
||||
void open(int port, int maxBufferSize) throws IOException;
|
||||
|
||||
int getPort();
|
||||
|
||||
void stop();
|
||||
|
||||
void close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads from the Datagram channel into an available buffer. If data is read then the buffer is queued for
|
||||
* processing, otherwise the buffer is returned to the buffer pool.
|
||||
*/
|
||||
private static class DatagramChannelReader implements ChannelReader {
|
||||
|
||||
private final BlockingQueue<ByteBuffer> bufferPool;
|
||||
private final BlockingQueue<RawSyslogEvent> syslogEvents;
|
||||
private final ProcessorLog logger;
|
||||
private DatagramChannel datagramChannel;
|
||||
private volatile boolean stopped = false;
|
||||
private Selector selector;
|
||||
|
||||
public DatagramChannelReader(final BlockingQueue<ByteBuffer> bufferPool, final BlockingQueue<RawSyslogEvent> syslogEvents, final ProcessorLog logger) {
|
||||
this.bufferPool = bufferPool;
|
||||
this.syslogEvents = syslogEvents;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(final int port, int maxBufferSize) throws IOException {
|
||||
datagramChannel = DatagramChannel.open();
|
||||
datagramChannel.configureBlocking(false);
|
||||
if (maxBufferSize > 0) {
|
||||
datagramChannel.setOption(StandardSocketOptions.SO_RCVBUF, maxBufferSize);
|
||||
final int actualReceiveBufSize = datagramChannel.getOption(StandardSocketOptions.SO_RCVBUF);
|
||||
if (actualReceiveBufSize < maxBufferSize) {
|
||||
logMaxBufferWarning(logger, maxBufferSize, actualReceiveBufSize);
|
||||
}
|
||||
}
|
||||
datagramChannel.socket().bind(new InetSocketAddress(port));
|
||||
selector = Selector.open();
|
||||
datagramChannel.register(selector, SelectionKey.OP_READ);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
final ByteBuffer buffer = bufferPool.poll();
|
||||
while (!stopped) {
|
||||
try {
|
||||
int selected = selector.select();
|
||||
if (selected > 0){
|
||||
Iterator<SelectionKey> selectorKeys = selector.selectedKeys().iterator();
|
||||
while (selectorKeys.hasNext()) {
|
||||
SelectionKey key = selectorKeys.next();
|
||||
selectorKeys.remove();
|
||||
if (!key.isValid()) {
|
||||
continue;
|
||||
}
|
||||
DatagramChannel channel = (DatagramChannel) key.channel();
|
||||
SocketAddress socketAddress;
|
||||
buffer.clear();
|
||||
while (!stopped && (socketAddress = channel.receive(buffer)) != null) {
|
||||
String sender = "";
|
||||
if (socketAddress instanceof InetSocketAddress) {
|
||||
sender = ((InetSocketAddress) socketAddress).getAddress().toString();
|
||||
}
|
||||
|
||||
// create a byte array from the buffer
|
||||
buffer.flip();
|
||||
byte bytes[] = new byte[buffer.limit()];
|
||||
buffer.get(bytes, 0, buffer.limit());
|
||||
|
||||
// queue the raw message with the sender, block until space is available
|
||||
syslogEvents.put(new RawSyslogEvent(bytes, sender));
|
||||
buffer.clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
stopped = true;
|
||||
} catch (IOException e) {
|
||||
logger.error("Error reading from DatagramChannel", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (buffer != null) {
|
||||
try {
|
||||
bufferPool.put(buffer);
|
||||
} catch (InterruptedException e) {
|
||||
// nothing to do here
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPort() {
|
||||
return datagramChannel == null ? 0 : datagramChannel.socket().getLocalPort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
selector.wakeup();
|
||||
stopped = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
IOUtils.closeQuietly(selector);
|
||||
IOUtils.closeQuietly(datagramChannel);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Accepts Socket connections on the given port and creates a handler for each connection to
|
||||
* be executed by a thread pool.
|
||||
*/
|
||||
private static class SocketChannelDispatcher implements ChannelReader {
|
||||
|
||||
private final BlockingQueue<ByteBuffer> bufferPool;
|
||||
private final BlockingQueue<RawSyslogEvent> syslogEvents;
|
||||
private final ProcessorLog logger;
|
||||
private final ExecutorService executor;
|
||||
private volatile boolean stopped = false;
|
||||
private Selector selector;
|
||||
private final BlockingQueue<SelectionKey> keyQueue;
|
||||
private final int maxConnections;
|
||||
private final AtomicInteger currentConnections = new AtomicInteger(0);
|
||||
private final SSLContext sslContext;
|
||||
|
||||
public SocketChannelDispatcher(final BlockingQueue<ByteBuffer> bufferPool, final BlockingQueue<RawSyslogEvent> syslogEvents,
|
||||
final ProcessorLog logger, final int maxConnections, final SSLContext sslContext) {
|
||||
this.bufferPool = bufferPool;
|
||||
this.syslogEvents = syslogEvents;
|
||||
this.logger = logger;
|
||||
this.maxConnections = maxConnections;
|
||||
this.keyQueue = new LinkedBlockingQueue<>(maxConnections);
|
||||
this.sslContext = sslContext;
|
||||
this.executor = Executors.newFixedThreadPool(maxConnections);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(final int port, int maxBufferSize) throws IOException {
|
||||
final ServerSocketChannel serverSocketChannel = ServerSocketChannel.open();
|
||||
serverSocketChannel.configureBlocking(false);
|
||||
if (maxBufferSize > 0) {
|
||||
serverSocketChannel.setOption(StandardSocketOptions.SO_RCVBUF, maxBufferSize);
|
||||
final int actualReceiveBufSize = serverSocketChannel.getOption(StandardSocketOptions.SO_RCVBUF);
|
||||
if (actualReceiveBufSize < maxBufferSize) {
|
||||
logMaxBufferWarning(logger, maxBufferSize, actualReceiveBufSize);
|
||||
}
|
||||
}
|
||||
serverSocketChannel.socket().bind(new InetSocketAddress(port));
|
||||
selector = Selector.open();
|
||||
serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (!stopped) {
|
||||
try {
|
||||
int selected = selector.select();
|
||||
if (selected > 0){
|
||||
Iterator<SelectionKey> selectorKeys = selector.selectedKeys().iterator();
|
||||
while (selectorKeys.hasNext()){
|
||||
SelectionKey key = selectorKeys.next();
|
||||
selectorKeys.remove();
|
||||
if (!key.isValid()){
|
||||
continue;
|
||||
}
|
||||
if (key.isAcceptable()) {
|
||||
// Handle new connections coming in
|
||||
final ServerSocketChannel channel = (ServerSocketChannel) key.channel();
|
||||
final SocketChannel socketChannel = channel.accept();
|
||||
// Check for available connections
|
||||
if (currentConnections.incrementAndGet() > maxConnections){
|
||||
currentConnections.decrementAndGet();
|
||||
logger.warn("Rejecting connection from {} because max connections has been met",
|
||||
new Object[]{ socketChannel.getRemoteAddress().toString() });
|
||||
IOUtils.closeQuietly(socketChannel);
|
||||
continue;
|
||||
}
|
||||
logger.debug("Accepted incoming connection from {}",
|
||||
new Object[]{socketChannel.getRemoteAddress().toString()});
|
||||
// Set socket to non-blocking, and register with selector
|
||||
socketChannel.configureBlocking(false);
|
||||
SelectionKey readKey = socketChannel.register(selector, SelectionKey.OP_READ);
|
||||
|
||||
// Prepare the byte buffer for the reads, clear it out
|
||||
ByteBuffer buffer = bufferPool.poll();
|
||||
buffer.clear();
|
||||
buffer.mark();
|
||||
|
||||
// If we have an SSLContext then create an SSLEngine for the channel
|
||||
SSLEngine sslEngine = null;
|
||||
if (sslContext != null) {
|
||||
sslEngine = sslContext.createSSLEngine();
|
||||
}
|
||||
|
||||
// Attach the buffer and SSLEngine to the key
|
||||
SocketChannelAttachment attachment = new SocketChannelAttachment(buffer, sslEngine);
|
||||
readKey.attach(attachment);
|
||||
} else if (key.isReadable()) {
|
||||
// Clear out the operations the select is interested in until done reading
|
||||
key.interestOps(0);
|
||||
// Create a handler based on whether an SSLEngine was provided or not
|
||||
final Runnable handler;
|
||||
if (sslContext != null) {
|
||||
handler = new SSLSocketChannelHandler(key, this, syslogEvents, logger);
|
||||
} else {
|
||||
handler = new SocketChannelHandler(key, this, syslogEvents, logger);
|
||||
}
|
||||
// run the handler
|
||||
executor.execute(handler);
|
||||
}
|
||||
}
|
||||
}
|
||||
// Add back all idle sockets to the select
|
||||
SelectionKey key;
|
||||
while((key = keyQueue.poll()) != null){
|
||||
key.interestOps(SelectionKey.OP_READ);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.error("Error accepting connection from SocketChannel", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPort() {
|
||||
// Return the port for the key listening for accepts
|
||||
for(SelectionKey key : selector.keys()){
|
||||
if (key.isValid()) {
|
||||
final Channel channel = key.channel();
|
||||
if (channel instanceof ServerSocketChannel) {
|
||||
return ((ServerSocketChannel)channel).socket().getLocalPort();
|
||||
}
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
stopped = true;
|
||||
selector.wakeup();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// Wait a while for existing tasks to terminate
|
||||
if (!executor.awaitTermination(1000L, TimeUnit.MILLISECONDS)) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
} catch (InterruptedException ie) {
|
||||
// (Re-)Cancel if current thread also interrupted
|
||||
executor.shutdownNow();
|
||||
// Preserve interrupt status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
for(SelectionKey key : selector.keys()){
|
||||
IOUtils.closeQuietly(key.channel());
|
||||
}
|
||||
IOUtils.closeQuietly(selector);
|
||||
}
|
||||
|
||||
public void completeConnection(SelectionKey key) {
|
||||
// connection is done. Return the buffer to the pool
|
||||
try {
|
||||
SocketChannelAttachment attachment = (SocketChannelAttachment) key.attachment();
|
||||
bufferPool.put(attachment.getByteBuffer());
|
||||
} catch (InterruptedException e) {
|
||||
// nothing to do here
|
||||
}
|
||||
currentConnections.decrementAndGet();
|
||||
}
|
||||
|
||||
public void addBackForSelection(SelectionKey key) {
|
||||
keyQueue.offer(key);
|
||||
selector.wakeup();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads from the given SocketChannel into the provided buffer. If data is read then the buffer is queued for
|
||||
* processing, otherwise the buffer is returned to the buffer pool.
|
||||
*/
|
||||
private static class SocketChannelHandler implements Runnable {
|
||||
|
||||
private final SelectionKey key;
|
||||
private final SocketChannelDispatcher dispatcher;
|
||||
private final BlockingQueue<RawSyslogEvent> syslogEvents;
|
||||
private final ProcessorLog logger;
|
||||
private final ByteArrayOutputStream currBytes = new ByteArrayOutputStream(4096);
|
||||
|
||||
public SocketChannelHandler(final SelectionKey key, final SocketChannelDispatcher dispatcher, final BlockingQueue<RawSyslogEvent> syslogEvents, final ProcessorLog logger) {
|
||||
this.key = key;
|
||||
this.dispatcher = dispatcher;
|
||||
this.syslogEvents = syslogEvents;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
boolean eof = false;
|
||||
SocketChannel socketChannel = null;
|
||||
|
||||
try {
|
||||
int bytesRead;
|
||||
socketChannel = (SocketChannel) key.channel();
|
||||
|
||||
SocketChannelAttachment attachment = (SocketChannelAttachment) key.attachment();
|
||||
ByteBuffer socketBuffer = attachment.getByteBuffer();
|
||||
|
||||
// read until the buffer is full
|
||||
while ((bytesRead = socketChannel.read(socketBuffer)) > 0) {
|
||||
// prepare byte buffer for reading
|
||||
socketBuffer.flip();
|
||||
// mark the current position as start, in case of partial message read
|
||||
socketBuffer.mark();
|
||||
|
||||
// get total bytes in buffer
|
||||
int total = socketBuffer.remaining();
|
||||
// go through the buffer looking for the end of each message
|
||||
currBytes.reset();
|
||||
for (int i = 0; i < total; i++) {
|
||||
// NOTE: For higher throughput, the looking for \n and copying into the byte
|
||||
// stream could be improved
|
||||
// Pull data out of buffer and cram into byte array
|
||||
byte currByte = socketBuffer.get();
|
||||
currBytes.write(currByte);
|
||||
|
||||
// check if at end of a message
|
||||
if (currByte == '\n') {
|
||||
String sender = socketChannel.socket().getInetAddress().toString();
|
||||
// queue the raw event blocking until space is available, reset the buffer
|
||||
syslogEvents.put(new RawSyslogEvent(currBytes.toByteArray(), sender));
|
||||
currBytes.reset();
|
||||
// Mark this as the start of the next message
|
||||
socketBuffer.mark();
|
||||
}
|
||||
}
|
||||
// Preserve bytes in buffer for next call to run
|
||||
// NOTE: This code could benefit from the two ByteBuffer read calls to avoid
|
||||
// this compact for higher throughput
|
||||
socketBuffer.reset();
|
||||
socketBuffer.compact();
|
||||
logger.debug("done handling SocketChannel");
|
||||
}
|
||||
// Check for closed socket
|
||||
if( bytesRead < 0 ){
|
||||
eof = true;
|
||||
}
|
||||
} catch (ClosedByInterruptException | InterruptedException e) {
|
||||
logger.debug("read loop interrupted, closing connection");
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} catch (IOException e) {
|
||||
logger.error("Error reading from channel due to {}", new Object[] {e.getMessage()}, e);
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} finally {
|
||||
if(eof == true) {
|
||||
IOUtils.closeQuietly(socketChannel);
|
||||
dispatcher.completeConnection(key);
|
||||
} else {
|
||||
dispatcher.addBackForSelection(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wraps a SocketChannel with an SSLSocketChannel for receiving messages over TLS.
|
||||
*/
|
||||
private static class SSLSocketChannelHandler implements Runnable {
|
||||
|
||||
private final SelectionKey key;
|
||||
private final SocketChannelDispatcher dispatcher;
|
||||
private final BlockingQueue<RawSyslogEvent> syslogEvents;
|
||||
private final ProcessorLog logger;
|
||||
private final ByteArrayOutputStream currBytes = new ByteArrayOutputStream(4096);
|
||||
|
||||
public SSLSocketChannelHandler(final SelectionKey key, final SocketChannelDispatcher dispatcher, final BlockingQueue<RawSyslogEvent> syslogEvents, final ProcessorLog logger) {
|
||||
this.key = key;
|
||||
this.dispatcher = dispatcher;
|
||||
this.syslogEvents = syslogEvents;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
boolean eof = false;
|
||||
SSLSocketChannel sslSocketChannel = null;
|
||||
try {
|
||||
int bytesRead;
|
||||
final SocketChannel socketChannel = (SocketChannel) key.channel();
|
||||
final SocketChannelAttachment attachment = (SocketChannelAttachment) key.attachment();
|
||||
|
||||
// wrap the SocketChannel with an SSLSocketChannel using the SSLEngine from the attachment
|
||||
sslSocketChannel = new SSLSocketChannel(attachment.getSslEngine(), socketChannel, false);
|
||||
|
||||
// SSLSocketChannel deals with byte[] so ByteBuffer isn't used here, but we'll use the size to create a new byte[]
|
||||
final ByteBuffer socketBuffer = attachment.getByteBuffer();
|
||||
byte[] socketBufferArray = new byte[socketBuffer.limit()];
|
||||
|
||||
// read until no more data
|
||||
while ((bytesRead = sslSocketChannel.read(socketBufferArray)) > 0) {
|
||||
// go through the buffer looking for the end of each message
|
||||
for (int i = 0; i < bytesRead; i++) {
|
||||
final byte currByte = socketBufferArray[i];
|
||||
currBytes.write(currByte);
|
||||
|
||||
// check if at end of a message
|
||||
if (currByte == '\n') {
|
||||
final String sender = socketChannel.socket().getInetAddress().toString();
|
||||
// queue the raw event blocking until space is available, reset the temporary buffer
|
||||
syslogEvents.put(new RawSyslogEvent(currBytes.toByteArray(), sender));
|
||||
currBytes.reset();
|
||||
}
|
||||
}
|
||||
logger.debug("done handling SocketChannel");
|
||||
}
|
||||
|
||||
// Check for closed socket
|
||||
if( bytesRead < 0 ){
|
||||
eof = true;
|
||||
}
|
||||
} catch (ClosedByInterruptException | InterruptedException e) {
|
||||
logger.debug("read loop interrupted, closing connection");
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} catch (IOException e) {
|
||||
logger.error("Error reading from channel due to {}", new Object[] {e.getMessage()}, e);
|
||||
// Treat same as closed socket
|
||||
eof = true;
|
||||
} finally {
|
||||
if(eof == true) {
|
||||
IOUtils.closeQuietly(sslSocketChannel);
|
||||
dispatcher.completeConnection(key);
|
||||
} else {
|
||||
dispatcher.addBackForSelection(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static void logMaxBufferWarning(final ProcessorLog logger, int maxBufferSize, int actualReceiveBufSize) {
|
||||
logger.warn("Attempted to set Socket Buffer Size to " + maxBufferSize + " bytes but could only set to "
|
||||
+ actualReceiveBufSize + "bytes. You may want to consider changing the Operating System's "
|
||||
+ "maximum receive buffer");
|
||||
}
|
||||
|
||||
// Wrapper class to pass around the raw message and the host/ip that sent it
|
||||
static class RawSyslogEvent {
|
||||
static class RawSyslogEvent<C extends SelectableChannel> implements Event<C> {
|
||||
|
||||
final byte[] rawMessage;
|
||||
final String sender;
|
||||
|
||||
public RawSyslogEvent(byte[] rawMessage, String sender) {
|
||||
public RawSyslogEvent(final byte[] rawMessage, final String sender) {
|
||||
this.rawMessage = rawMessage;
|
||||
this.sender = sender;
|
||||
}
|
||||
|
||||
public byte[] getRawMessage() {
|
||||
@Override
|
||||
public byte[] getData() {
|
||||
return this.rawMessage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSender() {
|
||||
return this.sender;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelResponder getResponder() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
// Wrapper class so we can attach a buffer and/or an SSLEngine to the selector key
|
||||
private static class SocketChannelAttachment {
|
||||
/**
|
||||
* EventFactory implementation for RawSyslogEvent.
|
||||
*/
|
||||
private static class RawSyslogEventFactory implements EventFactory<RawSyslogEvent> {
|
||||
|
||||
private final ByteBuffer byteBuffer;
|
||||
private final SSLEngine sslEngine;
|
||||
|
||||
public SocketChannelAttachment(ByteBuffer byteBuffer, SSLEngine sslEngine) {
|
||||
this.byteBuffer = byteBuffer;
|
||||
this.sslEngine = sslEngine;
|
||||
@Override
|
||||
public RawSyslogEvent create(byte[] data, Map<String, String> metadata, final ChannelResponder responder) {
|
||||
String sender = null;
|
||||
if (metadata != null && metadata.containsKey(EventFactory.SENDER_KEY)) {
|
||||
sender = metadata.get(EventFactory.SENDER_KEY);
|
||||
}
|
||||
return new RawSyslogEvent(data, sender);
|
||||
}
|
||||
|
||||
public ByteBuffer getByteBuffer() {
|
||||
return byteBuffer;
|
||||
}
|
||||
|
||||
public SSLEngine getSslEngine() {
|
||||
return sslEngine;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -46,9 +46,9 @@ 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.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.AbstractSyslogProcessor.SyslogAttributes;
|
||||
import org.apache.nifi.processors.standard.util.SyslogEvent;
|
||||
import org.apache.nifi.processors.standard.util.SyslogParser;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogAttributes;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogEvent;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogParser;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ 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.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.SyslogParser;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogParser;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
import org.apache.nifi.util.ObjectHolder;
|
||||
|
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.relp.event;
|
||||
|
||||
import org.apache.nifi.processor.util.listen.event.StandardEvent;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
|
||||
import java.nio.channels.SocketChannel;
|
||||
|
||||
/**
|
||||
* A RELP event which adds the transaction number and command to the StandardEvent.
|
||||
*/
|
||||
public class RELPEvent extends StandardEvent<SocketChannel> {
|
||||
|
||||
private final long txnr;
|
||||
private final String command;
|
||||
|
||||
public RELPEvent(final String sender, final byte[] data, final ChannelResponder<SocketChannel> responder, final long txnr, final String command) {
|
||||
super(sender, data, responder);
|
||||
this.txnr = txnr;
|
||||
this.command = command;
|
||||
}
|
||||
|
||||
public long getTxnr() {
|
||||
return txnr;
|
||||
}
|
||||
|
||||
public String getCommand() {
|
||||
return command;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* 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.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* An EventFactory implementation to create RELPEvents.
|
||||
*/
|
||||
public class RELPEventFactory implements EventFactory<RELPEvent> {
|
||||
|
||||
@Override
|
||||
public RELPEvent create(final byte[] data, final Map<String, String> metadata, final ChannelResponder responder) {
|
||||
final long txnr = Long.valueOf(metadata.get(RELPMetadata.TXNR_KEY));
|
||||
final String command = metadata.get(RELPMetadata.COMMAND_KEY);
|
||||
final String sender = metadata.get(EventFactory.SENDER_KEY);
|
||||
return new RELPEvent(sender, data, responder, txnr, command);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* 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";
|
||||
|
||||
}
|
|
@ -0,0 +1,195 @@
|
|||
/*
|
||||
* 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.stream.io.ByteArrayOutputStream;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
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
|
||||
* @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 {}", new Object[]{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 {}", new Object[] {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 {}", new Object[] {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 {}", new Object[] {currBytes.size()});
|
||||
|
||||
if (currBytes.size() >= frameBuilder.dataLength) {
|
||||
final byte[] data = currBytes.toByteArray();
|
||||
frameBuilder.data(data);
|
||||
logger.debug("Reached expected data size of {}", new Object[] {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;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* 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();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,32 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.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);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* 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
|
||||
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* 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.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactoryUtil;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMetadata;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPEncoder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Encapsulates the logic to handle a RELPFrame once it has been read from the channel.
|
||||
*/
|
||||
public class RELPFrameHandler<E extends Event<SocketChannel>> {
|
||||
|
||||
static final String CMD_OPEN = "open";
|
||||
static final String CMD_CLOSE = "close";
|
||||
|
||||
private final Charset charset;
|
||||
private final EventFactory<E> eventFactory;
|
||||
private final BlockingQueue<E> events;
|
||||
private final SelectionKey key;
|
||||
private final AsyncChannelDispatcher dispatcher;
|
||||
private final RELPEncoder encoder;
|
||||
|
||||
public RELPFrameHandler(final SelectionKey selectionKey,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final AsyncChannelDispatcher dispatcher) {
|
||||
this.key = selectionKey;
|
||||
this.charset = charset;
|
||||
this.eventFactory = eventFactory;
|
||||
this.events = events;
|
||||
this.dispatcher = dispatcher;
|
||||
this.encoder = new RELPEncoder(charset);
|
||||
}
|
||||
|
||||
public void handle(final RELPFrame frame, final ChannelResponder<SocketChannel> responder, final String sender)
|
||||
throws IOException, InterruptedException {
|
||||
|
||||
// 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));
|
||||
responder.addResponse(response);
|
||||
responder.respond();
|
||||
} else if (CMD_CLOSE.equals(frame.getCommand())) {
|
||||
ChannelResponse response = new RELPChannelResponse(encoder, RELPResponse.ok(frame.getTxnr()));
|
||||
responder.addResponse(response);
|
||||
responder.respond();
|
||||
dispatcher.completeConnection(key);
|
||||
} else {
|
||||
final Map<String, String> metadata = EventFactoryUtil.createMapWithSender(sender.toString());
|
||||
metadata.put(RELPMetadata.TXNR_KEY, String.valueOf(frame.getTxnr()));
|
||||
metadata.put(RELPMetadata.COMMAND_KEY, frame.getCommand());
|
||||
|
||||
// queue the raw event blocking until space is available, reset the buffer
|
||||
final E event = eventFactory.create(frame.getData(), metadata, responder);
|
||||
events.put(event);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.socket.SSLSocketChannelHandler;
|
||||
import org.apache.nifi.processor.util.listen.response.socket.SSLSocketChannelResponder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPDecoder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrameException;
|
||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* A RELP implementation of SSLSocketChannelHandler.
|
||||
*/
|
||||
public class RELPSSLSocketChannelHandler<E extends Event<SocketChannel>> extends SSLSocketChannelHandler<E> {
|
||||
|
||||
private RELPDecoder decoder;
|
||||
private RELPFrameHandler<E> frameHandler;
|
||||
|
||||
public RELPSSLSocketChannelHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
super(key, dispatcher, charset, eventFactory, events, logger);
|
||||
this.decoder = new RELPDecoder(charset);
|
||||
this.frameHandler = new RELPFrameHandler<>(key, charset, eventFactory, events, dispatcher);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processBuffer(final SSLSocketChannel sslSocketChannel, final SocketChannel socketChannel,
|
||||
final int bytesRead, final byte[] buffer) throws InterruptedException, IOException {
|
||||
|
||||
final InetAddress sender = socketChannel.socket().getInetAddress();
|
||||
try {
|
||||
// go through the buffer parsing the RELP command
|
||||
for (int i = 0; i < bytesRead; i++) {
|
||||
byte currByte = buffer[i];
|
||||
|
||||
// if we found the end of a frame, handle the frame and mark the buffer
|
||||
if (decoder.process(currByte)) {
|
||||
final RELPFrame frame = decoder.getFrame();
|
||||
|
||||
logger.debug("Received RELP frame with transaction {} and command {}",
|
||||
new Object[] {frame.getTxnr(), frame.getCommand()});
|
||||
|
||||
final SSLSocketChannelResponder responder = new SSLSocketChannelResponder(socketChannel, sslSocketChannel);
|
||||
frameHandler.handle(frame, responder, sender.toString());
|
||||
}
|
||||
}
|
||||
|
||||
logger.debug("Done processing buffer");
|
||||
|
||||
} catch (final RELPFrameException rfe) {
|
||||
logger.error("Error reading RELP frames due to {}", new Object[] {rfe.getMessage()} , rfe);
|
||||
// if an invalid frame or bad data was sent then the decoder will be left in a
|
||||
// corrupted state, so lets close the connection and cause the client to re-establish
|
||||
dispatcher.completeConnection(key);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,98 @@
|
|||
/*
|
||||
* 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.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.socket.StandardSocketChannelHandler;
|
||||
import org.apache.nifi.processor.util.listen.response.socket.SocketChannelResponder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPDecoder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrameException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Extends the StandardSocketChannelHandler to decode bytes into RELP frames.
|
||||
*/
|
||||
public class RELPSocketChannelHandler<E extends Event<SocketChannel>> extends StandardSocketChannelHandler<E> {
|
||||
|
||||
private RELPDecoder decoder;
|
||||
private RELPFrameHandler<E> frameHandler;
|
||||
|
||||
public RELPSocketChannelHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
super(key, dispatcher, charset, eventFactory, events, logger);
|
||||
this.decoder = new RELPDecoder(charset);
|
||||
this.frameHandler = new RELPFrameHandler<>(key, charset, eventFactory, events, dispatcher);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processBuffer(final SocketChannel socketChannel, final ByteBuffer socketBuffer)
|
||||
throws InterruptedException, IOException {
|
||||
|
||||
// get total bytes in buffer
|
||||
final int total = socketBuffer.remaining();
|
||||
final InetAddress sender = socketChannel.socket().getInetAddress();
|
||||
|
||||
try {
|
||||
// go through the buffer parsing the RELP command
|
||||
for (int i = 0; i < total; i++) {
|
||||
byte currByte = socketBuffer.get();
|
||||
|
||||
// if we found the end of a frame, handle the frame and mark the buffer
|
||||
if (decoder.process(currByte)) {
|
||||
final RELPFrame frame = decoder.getFrame();
|
||||
|
||||
logger.debug("Received RELP frame with transaction {} and command {}",
|
||||
new Object[] {frame.getTxnr(), frame.getCommand()});
|
||||
|
||||
final SocketChannelResponder responder = new SocketChannelResponder(socketChannel);
|
||||
frameHandler.handle(frame, responder, sender.toString());
|
||||
socketBuffer.mark();
|
||||
}
|
||||
}
|
||||
|
||||
logger.debug("Done processing buffer");
|
||||
|
||||
} catch (final RELPFrameException rfe) {
|
||||
logger.error("Error reading RELP frames due to {}", new Object[] {rfe.getMessage()}, rfe);
|
||||
// if an invalid frame or bad data was sent then the decoder will be left in a
|
||||
// corrupted state, so lets close the connection and cause the client to re-establish
|
||||
dispatcher.completeConnection(key);
|
||||
}
|
||||
}
|
||||
|
||||
// not used for anything in RELP since the decoder encapsulates the delimiter
|
||||
@Override
|
||||
public byte getDelimiter() {
|
||||
return RELPFrame.DELIMITER;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.relp.handler;
|
||||
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandler;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory;
|
||||
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
|
||||
/**
|
||||
* Default factory for creating RELP socket channel handlers.
|
||||
*/
|
||||
public class RELPSocketChannelHandlerFactory<E extends Event<SocketChannel>> implements ChannelHandlerFactory<E, AsyncChannelDispatcher> {
|
||||
|
||||
@Override
|
||||
public ChannelHandler<E, AsyncChannelDispatcher> createHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
return new RELPSocketChannelHandler<>(key, dispatcher, charset, eventFactory, events, logger);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelHandler<E, AsyncChannelDispatcher> createSSLHandler(final SelectionKey key,
|
||||
final AsyncChannelDispatcher dispatcher,
|
||||
final Charset charset,
|
||||
final EventFactory<E> eventFactory,
|
||||
final BlockingQueue<E> events,
|
||||
final ProcessorLog logger) {
|
||||
return new RELPSSLSocketChannelHandler<>(key, dispatcher, charset, eventFactory, events, logger);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.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);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,162 @@
|
|||
/*
|
||||
* 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 new RELPResponse(txnr, ERROR, "ERROR", 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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* 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.syslog;
|
||||
|
||||
import org.apache.nifi.flowfile.attributes.FlowFileAttributeKey;
|
||||
|
||||
/**
|
||||
* FlowFile Attributes for each Syslog message.
|
||||
*/
|
||||
public enum SyslogAttributes implements FlowFileAttributeKey {
|
||||
|
||||
PRIORITY("syslog.priority"),
|
||||
SEVERITY("syslog.severity"),
|
||||
FACILITY("syslog.facility"),
|
||||
VERSION("syslog.version"),
|
||||
TIMESTAMP("syslog.timestamp"),
|
||||
HOSTNAME("syslog.hostname"),
|
||||
SENDER("syslog.sender"),
|
||||
BODY("syslog.body"),
|
||||
VALID("syslog.valid"),
|
||||
PROTOCOL("syslog.protocol"),
|
||||
PORT("syslog.port");
|
||||
|
||||
private String key;
|
||||
|
||||
SyslogAttributes(String key) {
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String key() {
|
||||
return key;
|
||||
}
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.util;
|
||||
package org.apache.nifi.processors.standard.syslog;
|
||||
|
||||
/**
|
||||
* Encapsulates the parsed information for a single Syslog event.
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.util;
|
||||
package org.apache.nifi.processors.standard.syslog;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.Charset;
|
|
@ -46,6 +46,7 @@ org.apache.nifi.processors.standard.GetJMSQueue
|
|||
org.apache.nifi.processors.standard.GetJMSTopic
|
||||
org.apache.nifi.processors.standard.ListFile
|
||||
org.apache.nifi.processors.standard.ListenHTTP
|
||||
org.apache.nifi.processors.standard.ListenRELP
|
||||
org.apache.nifi.processors.standard.ListenSyslog
|
||||
org.apache.nifi.processors.standard.ListenUDP
|
||||
org.apache.nifi.processors.standard.ListSFTP
|
||||
|
|
|
@ -0,0 +1,241 @@
|
|||
/*
|
||||
* 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.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPEvent;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPEncoder;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.apache.nifi.processors.standard.relp.response.RELPResponse;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
import org.apache.nifi.ssl.StandardSSLContextService;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.IOException;
|
||||
import java.net.Socket;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
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 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();
|
||||
|
||||
private RELPEncoder encoder;
|
||||
private ResponseCapturingListenRELP proc;
|
||||
private TestRunner runner;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
encoder = new RELPEncoder(StandardCharsets.UTF_8);
|
||||
proc = new ResponseCapturingListenRELP();
|
||||
runner = TestRunners.newTestRunner(proc);
|
||||
runner.setProperty(ListenSyslog.PORT, "0");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testListenRELP() throws IOException, InterruptedException {
|
||||
final List<RELPFrame> frames = new ArrayList<>();
|
||||
frames.add(OPEN_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(CLOSE_FRAME);
|
||||
|
||||
// three syslog frames should be transferred and three responses should be sent
|
||||
run(frames, 3, 3, null);
|
||||
|
||||
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
|
||||
Assert.assertNotNull(events);
|
||||
Assert.assertEquals(3, events.size());
|
||||
|
||||
final ProvenanceEventRecord event = events.get(0);
|
||||
Assert.assertEquals(ProvenanceEventType.RECEIVE, event.getEventType());
|
||||
Assert.assertTrue("transit uri must be set and start with proper protocol", event.getTransitUri().toLowerCase().startsWith("relp"));
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenRELP.REL_SUCCESS);
|
||||
Assert.assertEquals(3, mockFlowFiles.size());
|
||||
|
||||
final MockFlowFile mockFlowFile = mockFlowFiles.get(0);
|
||||
Assert.assertEquals(String.valueOf(SYSLOG_FRAME.getTxnr()), mockFlowFile.getAttribute(ListenRELP.RELPAttributes.TXNR.key()));
|
||||
Assert.assertEquals(SYSLOG_FRAME.getCommand(), mockFlowFile.getAttribute(ListenRELP.RELPAttributes.COMMAND.key()));
|
||||
Assert.assertTrue(!StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.PORT.key())));
|
||||
Assert.assertTrue(!StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.SENDER.key())));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBatching() throws IOException, InterruptedException {
|
||||
runner.setProperty(ListenRELP.MAX_BATCH_SIZE, "5");
|
||||
|
||||
final List<RELPFrame> frames = new ArrayList<>();
|
||||
frames.add(OPEN_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(CLOSE_FRAME);
|
||||
|
||||
// one syslog frame should be transferred since we are batching, but three responses should be sent
|
||||
run(frames, 1, 3, null);
|
||||
|
||||
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
|
||||
Assert.assertNotNull(events);
|
||||
Assert.assertEquals(1, events.size());
|
||||
|
||||
final ProvenanceEventRecord event = events.get(0);
|
||||
Assert.assertEquals(ProvenanceEventType.RECEIVE, event.getEventType());
|
||||
Assert.assertTrue("transit uri must be set and start with proper protocol", event.getTransitUri().toLowerCase().startsWith("relp"));
|
||||
|
||||
final List<MockFlowFile> mockFlowFiles = runner.getFlowFilesForRelationship(ListenRELP.REL_SUCCESS);
|
||||
Assert.assertEquals(1, mockFlowFiles.size());
|
||||
|
||||
final MockFlowFile mockFlowFile = mockFlowFiles.get(0);
|
||||
Assert.assertEquals(SYSLOG_FRAME.getCommand(), mockFlowFile.getAttribute(ListenRELP.RELPAttributes.COMMAND.key()));
|
||||
Assert.assertTrue(!StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.PORT.key())));
|
||||
Assert.assertTrue(!StringUtils.isBlank(mockFlowFile.getAttribute(ListenRELP.RELPAttributes.SENDER.key())));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTLS() throws InitializationException, IOException, InterruptedException {
|
||||
final SSLContextService sslContextService = new StandardSSLContextService();
|
||||
runner.addControllerService("ssl-context", sslContextService);
|
||||
runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
|
||||
runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
|
||||
runner.setProperty(sslContextService, StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
|
||||
runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks");
|
||||
runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE_PASSWORD, "localtest");
|
||||
runner.setProperty(sslContextService, StandardSSLContextService.KEYSTORE_TYPE, "JKS");
|
||||
runner.enableControllerService(sslContextService);
|
||||
|
||||
runner.setProperty(PostHTTP.SSL_CONTEXT_SERVICE, "ssl-context");
|
||||
|
||||
final List<RELPFrame> frames = new ArrayList<>();
|
||||
frames.add(OPEN_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(SYSLOG_FRAME);
|
||||
frames.add(CLOSE_FRAME);
|
||||
|
||||
// three syslog frames should be transferred and three responses should be sent
|
||||
run(frames, 5, 5, sslContextService);
|
||||
}
|
||||
|
||||
protected void run(final List<RELPFrame> frames, final int expectedTransferred, final int expectedResponses, final SSLContextService sslContextService)
|
||||
throws IOException, InterruptedException {
|
||||
|
||||
Socket socket = null;
|
||||
try {
|
||||
// schedule to start listening on a random port
|
||||
final ProcessSessionFactory processSessionFactory = runner.getProcessSessionFactory();
|
||||
final ProcessContext context = runner.getProcessContext();
|
||||
proc.onScheduled(context);
|
||||
|
||||
// create a client connection to the port the dispatcher is listening on
|
||||
final int realPort = proc.getDispatcherPort();
|
||||
|
||||
// create either a regular socket or ssl socket based on context being passed in
|
||||
if (sslContextService != null) {
|
||||
final SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED);
|
||||
socket = sslContext.getSocketFactory().createSocket("localhost", realPort);
|
||||
} else {
|
||||
socket = new Socket("localhost", realPort);
|
||||
}
|
||||
Thread.sleep(100);
|
||||
|
||||
// send the frames to the port the processors is listening on
|
||||
sendFrames(frames, socket);
|
||||
|
||||
// call onTrigger until we processed all the frames, or a certain amount of time passes
|
||||
long responseTimeout = 10000;
|
||||
long startTime = System.currentTimeMillis();
|
||||
while (proc.responses.size() < expectedTransferred
|
||||
&& (System.currentTimeMillis() - startTime < responseTimeout)) {
|
||||
proc.onTrigger(context, processSessionFactory);
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
// should have gotten a response for each frame
|
||||
Assert.assertEquals(expectedResponses, proc.responses.size());
|
||||
|
||||
// should have transferred the expected events
|
||||
runner.assertTransferCount(ListenRELP.REL_SUCCESS, expectedTransferred);
|
||||
|
||||
} finally {
|
||||
// unschedule to close connections
|
||||
proc.onUnscheduled();
|
||||
IOUtils.closeQuietly(socket);
|
||||
}
|
||||
}
|
||||
|
||||
private void sendFrames(final List<RELPFrame> frames, final Socket socket) throws IOException, InterruptedException {
|
||||
// send the provided messages
|
||||
for (final RELPFrame frame : frames) {
|
||||
byte[] encodedFrame = encoder.encode(frame);
|
||||
socket.getOutputStream().write(encodedFrame);
|
||||
Thread.sleep(1);
|
||||
}
|
||||
socket.getOutputStream().flush();
|
||||
}
|
||||
|
||||
// Extend ListenRELP so we can use the CapturingSocketChannelResponseDispatcher
|
||||
private static class ResponseCapturingListenRELP extends ListenRELP {
|
||||
|
||||
private List<RELPResponse> responses = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
protected void respond(RELPEvent event, RELPResponse relpResponse) {
|
||||
this.responses.add(relpResponse);
|
||||
super.respond(event, relpResponse);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -16,7 +16,26 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.exception.FlowFileAccessException;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogAttributes;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogEvent;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogParser;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.util.IntegerHolder;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -29,26 +48,7 @@ import java.util.ArrayList;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.exception.FlowFileAccessException;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processors.standard.ListenSyslog.RawSyslogEvent;
|
||||
import org.apache.nifi.processors.standard.util.SyslogEvent;
|
||||
import org.apache.nifi.processors.standard.util.SyslogParser;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.util.IntegerHolder;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class TestListenSyslog {
|
||||
|
||||
|
@ -301,9 +301,9 @@ public class TestListenSyslog {
|
|||
runner.assertAllFlowFilesTransferred(ListenSyslog.REL_SUCCESS, 1);
|
||||
|
||||
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(ListenSyslog.REL_SUCCESS).get(0);
|
||||
Assert.assertEquals("0", flowFile.getAttribute(ListenSyslog.SyslogAttributes.PORT.key()));
|
||||
Assert.assertEquals(ListenSyslog.UDP_VALUE.getValue(), flowFile.getAttribute(ListenSyslog.SyslogAttributes.PROTOCOL.key()));
|
||||
Assert.assertTrue(!StringUtils.isBlank(flowFile.getAttribute(ListenSyslog.SyslogAttributes.SENDER.key())));
|
||||
Assert.assertEquals("0", flowFile.getAttribute(SyslogAttributes.PORT.key()));
|
||||
Assert.assertEquals(ListenSyslog.UDP_VALUE.getValue(), flowFile.getAttribute(SyslogAttributes.PROTOCOL.key()));
|
||||
Assert.assertTrue(!StringUtils.isBlank(flowFile.getAttribute(SyslogAttributes.SENDER.key())));
|
||||
|
||||
final String content = new String(flowFile.toByteArray(), StandardCharsets.UTF_8);
|
||||
final String[] splits = content.split("\\|");
|
||||
|
@ -391,16 +391,16 @@ public class TestListenSyslog {
|
|||
|
||||
@Test
|
||||
public void testErrorQueue() throws IOException {
|
||||
final List<RawSyslogEvent> msgs = new ArrayList<>();
|
||||
msgs.add(new RawSyslogEvent(VALID_MESSAGE.getBytes(), "sender-01"));
|
||||
msgs.add(new RawSyslogEvent(VALID_MESSAGE.getBytes(), "sender-01"));
|
||||
final List<ListenSyslog.RawSyslogEvent> msgs = new ArrayList<>();
|
||||
msgs.add(new ListenSyslog.RawSyslogEvent(VALID_MESSAGE.getBytes(), "sender-01"));
|
||||
msgs.add(new ListenSyslog.RawSyslogEvent(VALID_MESSAGE.getBytes(), "sender-01"));
|
||||
|
||||
// Add message that will throw a FlowFileAccessException the first time that we attempt to read
|
||||
// the contents but will succeeed the second time.
|
||||
// the contents but will succeed the second time.
|
||||
final IntegerHolder getMessageAttempts = new IntegerHolder(0);
|
||||
msgs.add(new RawSyslogEvent(VALID_MESSAGE.getBytes(), "sender-01") {
|
||||
msgs.add(new ListenSyslog.RawSyslogEvent(VALID_MESSAGE.getBytes(), "sender-01") {
|
||||
@Override
|
||||
public byte[] getRawMessage() {
|
||||
public byte[] getData() {
|
||||
final int attempts = getMessageAttempts.incrementAndGet();
|
||||
if (attempts == 1) {
|
||||
throw new FlowFileAccessException("Unit test failure");
|
||||
|
@ -432,16 +432,16 @@ public class TestListenSyslog {
|
|||
|
||||
private void checkFlowFile(final MockFlowFile flowFile, final int port, final String protocol) {
|
||||
flowFile.assertContentEquals(VALID_MESSAGE);
|
||||
Assert.assertEquals(PRI, flowFile.getAttribute(ListenSyslog.SyslogAttributes.PRIORITY.key()));
|
||||
Assert.assertEquals(SEV, flowFile.getAttribute(ListenSyslog.SyslogAttributes.SEVERITY.key()));
|
||||
Assert.assertEquals(FAC, flowFile.getAttribute(ListenSyslog.SyslogAttributes.FACILITY.key()));
|
||||
Assert.assertEquals(TIME, flowFile.getAttribute(ListenSyslog.SyslogAttributes.TIMESTAMP.key()));
|
||||
Assert.assertEquals(HOST, flowFile.getAttribute(ListenSyslog.SyslogAttributes.HOSTNAME.key()));
|
||||
Assert.assertEquals(BODY, flowFile.getAttribute(ListenSyslog.SyslogAttributes.BODY.key()));
|
||||
Assert.assertEquals("true", flowFile.getAttribute(ListenSyslog.SyslogAttributes.VALID.key()));
|
||||
Assert.assertEquals(String.valueOf(port), flowFile.getAttribute(ListenSyslog.SyslogAttributes.PORT.key()));
|
||||
Assert.assertEquals(protocol, flowFile.getAttribute(ListenSyslog.SyslogAttributes.PROTOCOL.key()));
|
||||
Assert.assertTrue(!StringUtils.isBlank(flowFile.getAttribute(ListenSyslog.SyslogAttributes.SENDER.key())));
|
||||
Assert.assertEquals(PRI, flowFile.getAttribute(SyslogAttributes.PRIORITY.key()));
|
||||
Assert.assertEquals(SEV, flowFile.getAttribute(SyslogAttributes.SEVERITY.key()));
|
||||
Assert.assertEquals(FAC, flowFile.getAttribute(SyslogAttributes.FACILITY.key()));
|
||||
Assert.assertEquals(TIME, flowFile.getAttribute(SyslogAttributes.TIMESTAMP.key()));
|
||||
Assert.assertEquals(HOST, flowFile.getAttribute(SyslogAttributes.HOSTNAME.key()));
|
||||
Assert.assertEquals(BODY, flowFile.getAttribute(SyslogAttributes.BODY.key()));
|
||||
Assert.assertEquals("true", flowFile.getAttribute(SyslogAttributes.VALID.key()));
|
||||
Assert.assertEquals(String.valueOf(port), flowFile.getAttribute(SyslogAttributes.PORT.key()));
|
||||
Assert.assertEquals(protocol, flowFile.getAttribute(SyslogAttributes.PROTOCOL.key()));
|
||||
Assert.assertTrue(!StringUtils.isBlank(flowFile.getAttribute(SyslogAttributes.SENDER.key())));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.nifi.processors.standard.AbstractSyslogProcessor.SyslogAttributes;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogAttributes;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
|
|
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* 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.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
import org.apache.nifi.processor.util.listen.response.socket.SocketChannelResponder;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestRELPEventFactory {
|
||||
|
||||
@Test
|
||||
public void testCreateRELPEvent() {
|
||||
final byte[] data = "this is an event".getBytes(StandardCharsets.UTF_8);
|
||||
|
||||
final String sender = "sender1";
|
||||
final long txnr = 1;
|
||||
final String command = "syslog";
|
||||
|
||||
final Map<String,String> metadata = new HashMap<>();
|
||||
metadata.put(EventFactory.SENDER_KEY, sender);
|
||||
metadata.put(RELPMetadata.TXNR_KEY, String.valueOf(txnr));
|
||||
metadata.put(RELPMetadata.COMMAND_KEY, command);
|
||||
|
||||
final ChannelResponder responder = new SocketChannelResponder(null);
|
||||
|
||||
final EventFactory<RELPEvent> factory = new RELPEventFactory();
|
||||
|
||||
final RELPEvent event = factory.create(data, metadata, responder);
|
||||
Assert.assertEquals(data, event.getData());
|
||||
Assert.assertEquals(sender, event.getSender());
|
||||
Assert.assertEquals(txnr, event.getTxnr());
|
||||
Assert.assertEquals(command, event.getCommand());
|
||||
Assert.assertEquals(responder, event.getResponder());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* 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.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
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;
|
||||
|
||||
@Before
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertNotNull(frame);
|
||||
Assert.assertEquals(1, frame.getTxnr());
|
||||
Assert.assertEquals("open", frame.getCommand());
|
||||
Assert.assertEquals(85, frame.getDataLength());
|
||||
|
||||
Assert.assertNotNull(frame.getData());
|
||||
Assert.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());
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertEquals(3, frames.size());
|
||||
|
||||
final RELPFrame frame1 = frames.get(0);
|
||||
Assert.assertNotNull(frame1);
|
||||
Assert.assertEquals(1, frame1.getTxnr());
|
||||
Assert.assertEquals("open", frame1.getCommand());
|
||||
Assert.assertEquals(85, frame1.getDataLength());
|
||||
|
||||
Assert.assertNotNull(frame1.getData());
|
||||
Assert.assertEquals(OPEN_FRAME_DATA, new String(frame1.getData(), StandardCharsets.UTF_8));
|
||||
|
||||
final RELPFrame frame2 = frames.get(1);
|
||||
Assert.assertNotNull(frame2);
|
||||
Assert.assertEquals(2, frame2.getTxnr());
|
||||
Assert.assertEquals("syslog", frame2.getCommand());
|
||||
Assert.assertEquals(29, frame2.getDataLength());
|
||||
|
||||
Assert.assertNotNull(frame2.getData());
|
||||
Assert.assertEquals(SYSLOG_FRAME_DATA, new String(frame2.getData(), StandardCharsets.UTF_8));
|
||||
|
||||
final RELPFrame frame3 = frames.get(2);
|
||||
Assert.assertNotNull(frame3);
|
||||
Assert.assertEquals(3, frame3.getTxnr());
|
||||
Assert.assertEquals("close", frame3.getCommand());
|
||||
Assert.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());
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertEquals(3, frames.size());
|
||||
}
|
||||
|
||||
@Test(expected = RELPFrameException.class)
|
||||
public void testBadDataShouldThrowException() throws RELPFrameException {
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
Assert.fail("Should have thrown exception");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestRELPEncoder {
|
||||
|
||||
@Test
|
||||
public void testEncodingWithData() throws IOException {
|
||||
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";
|
||||
Assert.assertEquals(expected, new String(result, StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEncodingNoData() throws IOException {
|
||||
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";
|
||||
Assert.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;
|
||||
}
|
||||
}
|
||||
|
||||
Assert.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));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.relp.frame;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRELPFrame {
|
||||
|
||||
@Test(expected = RELPFrameException.class)
|
||||
public void testInvalidTxnr() {
|
||||
new RELPFrame.Builder().command("command").dataLength(5).data(new byte[5]).build();
|
||||
}
|
||||
|
||||
@Test(expected = RELPFrameException.class)
|
||||
public void testInvalidCommand() {
|
||||
new RELPFrame.Builder().txnr(1).dataLength(5).data(new byte[5]).build();
|
||||
}
|
||||
|
||||
@Test(expected = RELPFrameException.class)
|
||||
public void testBlankCommand() {
|
||||
new RELPFrame.Builder().txnr(1).command(" ").dataLength(5).data(new byte[5]).build();
|
||||
}
|
||||
|
||||
@Test(expected = RELPFrameException.class)
|
||||
public void testInvalidDataLength() {
|
||||
new RELPFrame.Builder().txnr(1).command("command").data(new byte[5]).build();
|
||||
}
|
||||
|
||||
@Test(expected = RELPFrameException.class)
|
||||
public void testInvalidData() {
|
||||
new RELPFrame.Builder().txnr(1).command("command").dataLength(5).data(null).build();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,163 @@
|
|||
/*
|
||||
* 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.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponse;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPEvent;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPEventFactory;
|
||||
import org.apache.nifi.processors.standard.relp.frame.RELPFrame;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.SelectionKey;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
public class TestRELPFrameHandler {
|
||||
|
||||
private Charset charset;
|
||||
private EventFactory<RELPEvent> eventFactory;
|
||||
private BlockingQueue<RELPEvent> events;
|
||||
private SelectionKey key;
|
||||
private AsyncChannelDispatcher dispatcher;
|
||||
|
||||
private RELPFrameHandler<RELPEvent> frameHandler;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
this.charset = StandardCharsets.UTF_8;
|
||||
this.eventFactory = new RELPEventFactory();
|
||||
this.events = new LinkedBlockingQueue<>();
|
||||
this.key = Mockito.mock(SelectionKey.class);
|
||||
this.dispatcher = Mockito.mock(AsyncChannelDispatcher.class);
|
||||
|
||||
this.frameHandler = new RELPFrameHandler<>(key, charset, eventFactory, events, dispatcher);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOpen() throws IOException, InterruptedException {
|
||||
final String offer1 = "relp_version=0";
|
||||
final String offer2 = "relp_software=librelp,1.2.7,http://librelp.adiscon.com";
|
||||
final String offer3 = "commands=syslog";
|
||||
|
||||
final String data = offer1 + "\n" + offer2 + "\n" + offer3;
|
||||
|
||||
final RELPFrame openFrame = new RELPFrame.Builder()
|
||||
.txnr(1).command("open")
|
||||
.dataLength(data.length())
|
||||
.data(data.getBytes(charset))
|
||||
.build();
|
||||
|
||||
final String sender = "sender1";
|
||||
final CapturingChannelResponder responder = new CapturingChannelResponder();
|
||||
|
||||
// call the handler and verify respond() was called once with once response
|
||||
frameHandler.handle(openFrame, responder, sender);
|
||||
Assert.assertEquals(1, responder.responded);
|
||||
Assert.assertEquals(1, responder.responses.size());
|
||||
|
||||
// verify the response sent back the offers that were received
|
||||
final ChannelResponse response = responder.responses.get(0);
|
||||
final String responseData = new String(response.toByteArray(), charset);
|
||||
Assert.assertTrue(responseData.contains(offer1));
|
||||
Assert.assertTrue(responseData.contains(offer2));
|
||||
Assert.assertTrue(responseData.contains(offer3));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClose() throws IOException, InterruptedException {
|
||||
final RELPFrame openFrame = new RELPFrame.Builder()
|
||||
.txnr(1).command("close")
|
||||
.dataLength(0)
|
||||
.data(new byte[0])
|
||||
.build();
|
||||
|
||||
final String sender = "sender1";
|
||||
final CapturingChannelResponder responder = new CapturingChannelResponder();
|
||||
|
||||
// call the handler and verify respond() was called once with once response
|
||||
frameHandler.handle(openFrame, responder, sender);
|
||||
Assert.assertEquals(1, responder.responded);
|
||||
Assert.assertEquals(1, responder.responses.size());
|
||||
|
||||
// verify the response sent back the offers that were received
|
||||
final ChannelResponse response = responder.responses.get(0);
|
||||
final String responseData = new String(response.toByteArray(), charset);
|
||||
Assert.assertTrue(responseData.contains("200 OK"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommand() throws IOException, InterruptedException {
|
||||
final String data = "this is a syslog message";
|
||||
|
||||
final RELPFrame openFrame = new RELPFrame.Builder()
|
||||
.txnr(1).command("syslog")
|
||||
.dataLength(data.length())
|
||||
.data(data.getBytes(charset))
|
||||
.build();
|
||||
|
||||
final String sender = "sender1";
|
||||
final CapturingChannelResponder responder = new CapturingChannelResponder();
|
||||
|
||||
// call the handler and verify respond() was called once with once response
|
||||
frameHandler.handle(openFrame, responder, sender);
|
||||
Assert.assertEquals(0, responder.responded);
|
||||
Assert.assertEquals(0, responder.responses.size());
|
||||
Assert.assertEquals(1, events.size());
|
||||
|
||||
final RELPEvent event = events.poll();
|
||||
Assert.assertEquals(data, new String(event.getData(), charset));
|
||||
}
|
||||
|
||||
private static class CapturingChannelResponder implements ChannelResponder<SocketChannel> {
|
||||
|
||||
int responded;
|
||||
List<ChannelResponse> responses = new ArrayList<>();
|
||||
|
||||
@Override
|
||||
public SocketChannel getChannel() {
|
||||
return Mockito.mock(SocketChannel.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ChannelResponse> getResponses() {
|
||||
return responses;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addResponse(ChannelResponse response) {
|
||||
responses.add(response);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void respond() throws IOException {
|
||||
responded++;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,209 @@
|
|||
/*
|
||||
* 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.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.AsyncChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.ChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.dispatcher.SocketChannelDispatcher;
|
||||
import org.apache.nifi.processor.util.listen.event.Event;
|
||||
import org.apache.nifi.processor.util.listen.event.EventFactory;
|
||||
import org.apache.nifi.processor.util.listen.handler.ChannelHandlerFactory;
|
||||
import org.apache.nifi.processor.util.listen.response.ChannelResponder;
|
||||
import org.apache.nifi.processors.standard.relp.event.RELPMetadata;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
public class TestRELPSocketChannelHandler {
|
||||
|
||||
private EventFactory<TestEvent> eventFactory;
|
||||
private ChannelHandlerFactory<TestEvent,AsyncChannelDispatcher> channelHandlerFactory;
|
||||
private BlockingQueue<ByteBuffer> byteBuffers;
|
||||
private BlockingQueue<TestEvent> events;
|
||||
private ProcessorLog logger = Mockito.mock(ProcessorLog.class);
|
||||
private int maxConnections;
|
||||
private SSLContext sslContext;
|
||||
private Charset charset;
|
||||
private ChannelDispatcher dispatcher;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
eventFactory = new TestEventHolderFactory();
|
||||
channelHandlerFactory = new RELPSocketChannelHandlerFactory<>();
|
||||
|
||||
byteBuffers = new LinkedBlockingQueue<>();
|
||||
byteBuffers.add(ByteBuffer.allocate(4096));
|
||||
|
||||
events = new LinkedBlockingQueue<>();
|
||||
logger = Mockito.mock(ProcessorLog.class);
|
||||
|
||||
maxConnections = 1;
|
||||
sslContext = null;
|
||||
charset = StandardCharsets.UTF_8;
|
||||
|
||||
dispatcher = new SocketChannelDispatcher<>(eventFactory, channelHandlerFactory, byteBuffers, events, logger,
|
||||
maxConnections, sslContext, charset);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBasicHandling() throws IOException, InterruptedException {
|
||||
final List<String> messages = new ArrayList<>();
|
||||
messages.add("1 syslog 20 this is message 1234\n");
|
||||
messages.add("2 syslog 22 this is message 456789\n");
|
||||
messages.add("3 syslog 21 this is message ABCDE\n");
|
||||
|
||||
run(messages);
|
||||
Assert.assertEquals(messages.size(), events.size());
|
||||
|
||||
boolean found1 = false;
|
||||
boolean found2 = false;
|
||||
boolean found3 = false;
|
||||
|
||||
TestEvent event;
|
||||
while((event = events.poll()) != null) {
|
||||
Map<String,String> metadata = event.metadata;
|
||||
Assert.assertTrue(metadata.containsKey(RELPMetadata.TXNR_KEY));
|
||||
|
||||
final String txnr = metadata.get(RELPMetadata.TXNR_KEY);
|
||||
if (txnr.equals("1")) {
|
||||
found1 = true;
|
||||
} else if (txnr.equals("2")) {
|
||||
found2 = true;
|
||||
} else if (txnr.equals("3")) {
|
||||
found3 = true;
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertTrue(found1);
|
||||
Assert.assertTrue(found2);
|
||||
Assert.assertTrue(found3);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLotsOfFrames() throws IOException, InterruptedException {
|
||||
final String baseMessage = " syslog 19 this is message ";
|
||||
final List<String> messages = new ArrayList<>();
|
||||
|
||||
for (int i=100; i < 1000; i++) {
|
||||
messages.add(i + baseMessage + i + "\n");
|
||||
}
|
||||
|
||||
run(messages);
|
||||
Assert.assertEquals(messages.size(), events.size());
|
||||
}
|
||||
|
||||
protected void run(List<String> messages) throws IOException, InterruptedException {
|
||||
final ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||
try {
|
||||
// starts the dispatcher listening on port 0 so it selects a random port
|
||||
dispatcher.open(0, 4096);
|
||||
|
||||
// starts a thread to run the dispatcher which will accept/read connections
|
||||
Thread dispatcherThread = new Thread(dispatcher);
|
||||
dispatcherThread.start();
|
||||
|
||||
|
||||
// create a client connection to the port the dispatcher is listening on
|
||||
final int realPort = dispatcher.getPort();
|
||||
try (SocketChannel channel = SocketChannel.open()) {
|
||||
channel.connect(new InetSocketAddress("localhost", realPort));
|
||||
Thread.sleep(100);
|
||||
|
||||
// send the provided messages
|
||||
for (int i=0; i < messages.size(); i++) {
|
||||
buffer.clear();
|
||||
buffer.put(messages.get(i).getBytes(charset));
|
||||
buffer.flip();
|
||||
|
||||
while (buffer.hasRemaining()) {
|
||||
channel.write(buffer);
|
||||
}
|
||||
Thread.sleep(1);
|
||||
}
|
||||
}
|
||||
|
||||
// wait up to 10 seconds to verify the responses
|
||||
long timeout = 10000;
|
||||
long startTime = System.currentTimeMillis();
|
||||
while (events.size() < messages.size() && (System.currentTimeMillis() - startTime < timeout)) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
// should have gotten an event for each message sent
|
||||
Assert.assertEquals(messages.size(), events.size());
|
||||
|
||||
} finally {
|
||||
// stop the dispatcher thread and ensure we shut down handler threads
|
||||
dispatcher.stop();
|
||||
dispatcher.close();
|
||||
}
|
||||
}
|
||||
|
||||
// Test event to produce from the data
|
||||
private static class TestEvent implements Event<SocketChannel> {
|
||||
|
||||
private byte[] data;
|
||||
private Map<String,String> metadata;
|
||||
|
||||
public TestEvent(byte[] data, Map<String, String> metadata) {
|
||||
this.data = data;
|
||||
this.metadata = metadata;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSender() {
|
||||
return metadata.get(EventFactory.SENDER_KEY);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getData() {
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelResponder<SocketChannel> getResponder() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
// Factory to create test events and send responses for testing
|
||||
private static class TestEventHolderFactory implements EventFactory<TestEvent> {
|
||||
|
||||
@Override
|
||||
public TestEvent create(final byte[] data, final Map<String, String> metadata, final ChannelResponder responder) {
|
||||
return new TestEvent(data, metadata);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,169 @@
|
|||
/*
|
||||
* 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.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestRELPResponse {
|
||||
|
||||
@Test
|
||||
public void testResponseToFrame() throws IOException {
|
||||
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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + " " + message + "\n" + data;
|
||||
Assert.assertEquals(expected, result);
|
||||
Assert.assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResponseToFrameNoMessage() throws IOException {
|
||||
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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + "\n" + data;
|
||||
Assert.assertEquals(expected, result);
|
||||
Assert.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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + " " + message;
|
||||
Assert.assertEquals(expected, result);
|
||||
Assert.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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = code + "";
|
||||
Assert.assertEquals(expected, result);
|
||||
Assert.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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.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";
|
||||
Assert.assertTrue(result.equals(expected1) || result.equals(expected2));
|
||||
Assert.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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = RELPResponse.OK + " OK\n";
|
||||
Assert.assertEquals(expected, result);
|
||||
Assert.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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = RELPResponse.OK + " OK";
|
||||
Assert.assertEquals(expected, result);
|
||||
Assert.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);
|
||||
Assert.assertEquals(txnr, frame.getTxnr());
|
||||
Assert.assertEquals(RELPResponse.RSP_CMD, frame.getCommand());
|
||||
|
||||
final String result = new String(frame.getData(), StandardCharsets.UTF_8);
|
||||
final String expected = RELPResponse.ERROR + " ERROR";
|
||||
Assert.assertEquals(expected, result);
|
||||
Assert.assertEquals(expected.length(), frame.getDataLength());
|
||||
}
|
||||
}
|
|
@ -16,6 +16,8 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard.util;
|
||||
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogEvent;
|
||||
import org.apache.nifi.processors.standard.syslog.SyslogParser;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
|
Loading…
Reference in New Issue