You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2015/01/19 01:23:04 UTC
[1/5] incubator-nifi git commit: NIFI-282: Begin refactoring and
creating client
Repository: incubator-nifi
Updated Branches:
refs/heads/site-to-site-client [created] fdf758460
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
deleted file mode 100644
index d4b4f61..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ /dev/null
@@ -1,510 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol.socket;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.zip.CRC32;
-import java.util.zip.CheckedInputStream;
-import java.util.zip.CheckedOutputStream;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.remote.Peer;
-import org.apache.nifi.remote.PeerStatus;
-import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.remote.RemoteResourceFactory;
-import org.apache.nifi.remote.StandardVersionNegotiator;
-import org.apache.nifi.remote.VersionNegotiator;
-import org.apache.nifi.remote.codec.FlowFileCodec;
-import org.apache.nifi.remote.codec.StandardFlowFileCodec;
-import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.ProtocolException;
-import org.apache.nifi.remote.io.CompressionInputStream;
-import org.apache.nifi.remote.io.CompressionOutputStream;
-import org.apache.nifi.remote.protocol.ClientProtocol;
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-import org.apache.nifi.remote.protocol.RequestType;
-import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.StopWatch;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SocketClientProtocol implements ClientProtocol {
- private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1);
-
-
- private RemoteGroupPort port;
- private boolean useCompression;
-
- private String commsIdentifier;
- private boolean handshakeComplete = false;
-
- private final Logger logger = LoggerFactory.getLogger(SocketClientProtocol.class);
-
- private Response handshakeResponse = null;
- private boolean readyForFileTransfer = false;
- private String transitUriPrefix = null;
-
- private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
-
- public SocketClientProtocol() {
- }
-
- public void setPort(final RemoteGroupPort port) {
- this.port = port;
- this.useCompression = port.isUseCompression();
- }
-
- @Override
- public void handshake(final Peer peer) throws IOException, HandshakeException {
- if ( handshakeComplete ) {
- throw new IllegalStateException("Handshake has already been completed");
- }
- commsIdentifier = UUID.randomUUID().toString();
- logger.debug("{} handshaking with {}", this, peer);
-
- final Map<HandshakeProperty, String> properties = new HashMap<>();
- properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression));
- properties.put(HandshakeProperty.PORT_IDENTIFIER, port.getIdentifier());
- properties.put(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, String.valueOf(
- port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS)) );
-
- final CommunicationsSession commsSession = peer.getCommunicationsSession();
- commsSession.setTimeout(port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS));
- final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
-
- dos.writeUTF(commsIdentifier);
-
- if ( versionNegotiator.getVersion() >= 3 ) {
- dos.writeUTF(peer.getUrl());
- transitUriPrefix = peer.getUrl();
-
- if ( !transitUriPrefix.endsWith("/") ) {
- transitUriPrefix = transitUriPrefix + "/";
- }
- }
-
- dos.writeInt(properties.size());
- for ( final Map.Entry<HandshakeProperty, String> entry : properties.entrySet() ) {
- dos.writeUTF(entry.getKey().name());
- dos.writeUTF(entry.getValue());
- }
-
- dos.flush();
-
- try {
- handshakeResponse = Response.read(dis);
- } catch (final ProtocolException e) {
- throw new HandshakeException(e);
- }
-
- switch (handshakeResponse.getCode()) {
- case PORT_NOT_IN_VALID_STATE:
- case UNKNOWN_PORT:
- case PORTS_DESTINATION_FULL:
- break;
- case PROPERTIES_OK:
- readyForFileTransfer = true;
- break;
- default:
- logger.error("{} received unexpected response {} from {} when negotiating Codec", new Object[] {
- this, handshakeResponse, peer});
- peer.close();
- throw new HandshakeException("Received unexpected response " + handshakeResponse);
- }
-
- logger.debug("{} Finished handshake with {}", this, peer);
- handshakeComplete = true;
- }
-
- public boolean isReadyForFileTransfer() {
- return readyForFileTransfer;
- }
-
- public boolean isPortInvalid() {
- if ( !handshakeComplete ) {
- throw new IllegalStateException("Handshake has not completed successfully");
- }
- return handshakeResponse.getCode() == ResponseCode.PORT_NOT_IN_VALID_STATE;
- }
-
- public boolean isPortUnknown() {
- if ( !handshakeComplete ) {
- throw new IllegalStateException("Handshake has not completed successfully");
- }
- return handshakeResponse.getCode() == ResponseCode.UNKNOWN_PORT;
- }
-
- public boolean isDestinationFull() {
- if ( !handshakeComplete ) {
- throw new IllegalStateException("Handshake has not completed successfully");
- }
- return handshakeResponse.getCode() == ResponseCode.PORTS_DESTINATION_FULL;
- }
-
- @Override
- public Set<PeerStatus> getPeerStatuses(final Peer peer) throws IOException {
- if ( !handshakeComplete ) {
- throw new IllegalStateException("Handshake has not been performed");
- }
-
- logger.debug("{} Get Peer Statuses from {}", this, peer);
- final CommunicationsSession commsSession = peer.getCommunicationsSession();
- final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
-
- RequestType.REQUEST_PEER_LIST.writeRequestType(dos);
- dos.flush();
- final int numPeers = dis.readInt();
- final Set<PeerStatus> peers = new HashSet<>(numPeers);
- for (int i=0; i < numPeers; i++) {
- final String hostname = dis.readUTF();
- final int port = dis.readInt();
- final boolean secure = dis.readBoolean();
- final int flowFileCount = dis.readInt();
- peers.add(new PeerStatus(hostname, port, secure, flowFileCount));
- }
-
- logger.debug("{} Received {} Peer Statuses from {}", this, peers.size(), peer);
- return peers;
- }
-
- @Override
- public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
- if ( !handshakeComplete ) {
- throw new IllegalStateException("Handshake has not been performed");
- }
-
- logger.debug("{} Negotiating Codec with {}", this, peer);
- final CommunicationsSession commsSession = peer.getCommunicationsSession();
- final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
-
- RequestType.NEGOTIATE_FLOWFILE_CODEC.writeRequestType(dos);
-
- FlowFileCodec codec = new StandardFlowFileCodec();
- try {
- codec = (FlowFileCodec) RemoteResourceFactory.initiateResourceNegotiation(codec, dis, dos);
- } catch (HandshakeException e) {
- throw new ProtocolException(e.toString());
- }
- logger.debug("{} negotiated FlowFileCodec {} with {}", new Object[] {this, codec, commsSession});
-
- return codec;
- }
-
-
- @Override
- public void receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
- if ( !handshakeComplete ) {
- throw new IllegalStateException("Handshake has not been performed");
- }
- if ( !readyForFileTransfer ) {
- throw new IllegalStateException("Cannot receive files; handshake resolution was " + handshakeResponse);
- }
-
- logger.debug("{} Receiving FlowFiles from {}", this, peer);
- final CommunicationsSession commsSession = peer.getCommunicationsSession();
- final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
- String userDn = commsSession.getUserDn();
- if ( userDn == null ) {
- userDn = "none";
- }
-
- // Indicate that we would like to have some data
- RequestType.RECEIVE_FLOWFILES.writeRequestType(dos);
- dos.flush();
-
- // Determine if Peer will send us data or has no data to send us
- final Response dataAvailableCode = Response.read(dis);
- switch (dataAvailableCode.getCode()) {
- case MORE_DATA:
- logger.debug("{} {} Indicates that data is available", this, peer);
- break;
- case NO_MORE_DATA:
- logger.debug("{} No data available from {}", peer);
- return;
- default:
- throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
- }
-
- final StopWatch stopWatch = new StopWatch(true);
- final Set<FlowFile> flowFilesReceived = new HashSet<>();
- long bytesReceived = 0L;
- final CRC32 crc = new CRC32();
-
- // Peer has data. Decode the bytes into FlowFiles until peer says he's finished sending data.
- boolean continueTransaction = true;
- String calculatedCRC = "";
- while (continueTransaction) {
- final InputStream flowFileInputStream = useCompression ? new CompressionInputStream(dis) : dis;
- final CheckedInputStream checkedIn = new CheckedInputStream(flowFileInputStream, crc);
-
- final long startNanos = System.nanoTime();
- FlowFile flowFile = codec.decode(checkedIn, session);
- final long transmissionNanos = System.nanoTime() - startNanos;
- final long transmissionMillis = TimeUnit.MILLISECONDS.convert(transmissionNanos, TimeUnit.NANOSECONDS);
-
- final String sourceFlowFileIdentifier = flowFile.getAttribute(CoreAttributes.UUID.key());
- flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
-
- final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceFlowFileIdentifier;
- session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transmissionMillis);
-
- session.transfer(flowFile, Relationship.ANONYMOUS);
- bytesReceived += flowFile.getSize();
- flowFilesReceived.add(flowFile);
- logger.debug("{} Received {} from {}", this, flowFile, peer);
-
- final Response transactionCode = Response.read(dis);
- switch (transactionCode.getCode()) {
- case CONTINUE_TRANSACTION:
- logger.trace("{} Received ContinueTransaction indicator from {}", this, peer);
- break;
- case FINISH_TRANSACTION:
- logger.trace("{} Received FinishTransaction indicator from {}", this, peer);
- continueTransaction = false;
- calculatedCRC = String.valueOf(checkedIn.getChecksum().getValue());
- break;
- default:
- throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionCode);
- }
- }
-
- // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
- // to peer so that we can verify that the connection is still open. This is a two-phase commit,
- // which helps to prevent the chances of data duplication. Without doing this, we may commit the
- // session and then when we send the response back to the peer, the peer may have timed out and may not
- // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
- // Critical Section involved in this transaction so that rather than the Critical Section being the
- // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
- logger.trace("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
- ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
-
- final Response confirmTransactionResponse = Response.read(dis);
- logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
-
- switch (confirmTransactionResponse.getCode()) {
- case CONFIRM_TRANSACTION:
- break;
- case BAD_CHECKSUM:
- session.rollback();
- throw new IOException(this + " Received a BadChecksum response from peer " + peer);
- default:
- throw new ProtocolException(this + " Received unexpected Response from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
- }
-
- // Commit the session so that we have persisted the data
- session.commit();
-
- if ( context.getAvailableRelationships().isEmpty() ) {
- // Confirm that we received the data and the peer can now discard it but that the peer should not
- // send any more data for a bit
- logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
- ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos);
- } else {
- // Confirm that we received the data and the peer can now discard it
- logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
- ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
- }
-
- stopWatch.stop();
- final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
- final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
- final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
- final String dataSize = FormatUtils.formatDataSize(bytesReceived);
- logger.info("{} Successfully receveied {} ({}) from {} in {} milliseconds at a rate of {}", new Object[] {
- this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
- }
-
- @Override
- public void transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
- if ( !handshakeComplete ) {
- throw new IllegalStateException("Handshake has not been performed");
- }
- if ( !readyForFileTransfer ) {
- throw new IllegalStateException("Cannot transfer files; handshake resolution was " + handshakeResponse);
- }
-
- FlowFile flowFile = session.get();
- if ( flowFile == null ) {
- return;
- }
-
- logger.debug("{} Sending FlowFiles to {}", this, peer);
- final CommunicationsSession commsSession = peer.getCommunicationsSession();
- final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
- String userDn = commsSession.getUserDn();
- if ( userDn == null ) {
- userDn = "none";
- }
-
- // Indicate that we would like to have some data
- RequestType.SEND_FLOWFILES.writeRequestType(dos);
- dos.flush();
-
- final StopWatch stopWatch = new StopWatch(true);
- final CRC32 crc = new CRC32();
-
- long bytesSent = 0L;
- final Set<FlowFile> flowFilesSent = new HashSet<>();
- boolean continueTransaction = true;
- String calculatedCRC = "";
- final long startSendingNanos = System.nanoTime();
- while (continueTransaction) {
- final OutputStream flowFileOutputStream = useCompression ? new CompressionOutputStream(dos) : dos;
- logger.debug("{} Sending {} to {}", this, flowFile, peer);
-
- final CheckedOutputStream checkedOutStream = new CheckedOutputStream(flowFileOutputStream, crc);
-
- final long startNanos = System.nanoTime();
- flowFile = codec.encode(flowFile, session, checkedOutStream);
- final long transferNanos = System.nanoTime() - startNanos;
- final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
-
- // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
- // Otherwise, do NOT close it because we don't want to close the underlying stream
- // (CompressionOutputStream will not close the underlying stream when it's closed)
- if ( useCompression ) {
- checkedOutStream.close();
- }
-
- flowFilesSent.add(flowFile);
- bytesSent += flowFile.getSize();
- logger.debug("{} Sent {} to {}", this, flowFile, peer);
-
- final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
- session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transferMillis, false);
- session.remove(flowFile);
-
- final long sendingNanos = System.nanoTime() - startSendingNanos;
- if ( sendingNanos < BATCH_SEND_NANOS ) {
- flowFile = session.get();
- } else {
- flowFile = null;
- }
-
- continueTransaction = (flowFile != null);
- if ( continueTransaction ) {
- logger.debug("{} Sent CONTINUE_TRANSACTION indicator to {}", this, peer);
- ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
- } else {
- logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
- ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
-
- calculatedCRC = String.valueOf( checkedOutStream.getChecksum().getValue() );
- }
- }
-
- // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
- final Response transactionConfirmationResponse = Response.read(dis);
- if ( transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION ) {
- // Confirm checksum and echo back the confirmation.
- logger.trace("{} Received {} from {}", this, transactionConfirmationResponse, peer);
- final String receivedCRC = transactionConfirmationResponse.getMessage();
-
- if ( versionNegotiator.getVersion() > 3 ) {
- if ( !receivedCRC.equals(calculatedCRC) ) {
- ResponseCode.BAD_CHECKSUM.writeResponse(dos);
- session.rollback();
- throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as " + calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC + "; canceling transaction and rolling back session");
- }
- }
-
- ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
- } else {
- throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
- }
-
- final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
-
- final Response transactionResponse;
- try {
- transactionResponse = Response.read(dis);
- } catch (final IOException e) {
- logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator." +
- " It is unknown whether or not the peer successfully received/processed the data." +
- " Therefore, {} will be rolled back, possibly resulting in data duplication of {}",
- this, peer, session, flowFileDescription);
- session.rollback();
- throw e;
- }
-
- logger.debug("{} Received {} from {}", this, transactionResponse, peer);
- if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
- peer.penalize(port.getYieldPeriod(TimeUnit.MILLISECONDS));
- } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
- throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
- }
-
- // consume input stream entirely, ignoring its contents. If we
- // don't do this, the Connection will not be returned to the pool
- stopWatch.stop();
- final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
- final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
- final String dataSize = FormatUtils.formatDataSize(bytesSent);
-
- session.commit();
-
- logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
- this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
- }
-
- @Override
- public VersionNegotiator getVersionNegotiator() {
- return versionNegotiator;
- }
-
- @Override
- public void shutdown(final Peer peer) throws IOException {
- readyForFileTransfer = false;
- final CommunicationsSession commsSession = peer.getCommunicationsSession();
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
-
- logger.debug("{} Shutting down with {}", this, peer);
- // Indicate that we would like to have some data
- RequestType.SHUTDOWN.writeRequestType(dos);
- dos.flush();
- }
-
- @Override
- public String getResourceName() {
- return "SocketFlowFileProtocol";
- }
-
- @Override
- public String toString() {
- return "SocketClientProtocol[CommsID=" + commsIdentifier + "]";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index 5edd4f9..647b45c 100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@ -32,7 +32,6 @@ import java.util.zip.CRC32;
import java.util.zip.CheckedInputStream;
import java.util.zip.CheckedOutputStream;
-import org.apache.nifi.cluster.NodeInformant;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.flowfile.FlowFile;
@@ -47,6 +46,7 @@ import org.apache.nifi.remote.RemoteResourceFactory;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.StandardVersionNegotiator;
import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.exception.ProtocolException;
@@ -58,7 +58,6 @@ import org.apache.nifi.remote.protocol.ServerProtocol;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.StopWatch;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
index e074010..7474d38 100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
@@ -18,14 +18,14 @@ package org.apache.nifi.remote;
import org.apache.nifi.remote.StandardRemoteGroupPort;
import org.apache.nifi.remote.PeerStatus;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.remote.cluster.NodeInformation;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import org.apache.nifi.cluster.ClusterNodeInformation;
-import org.apache.nifi.cluster.NodeInformation;
import org.apache.nifi.connectable.ConnectableType;
-
import org.junit.Test;
public class TestStandardRemoteGroupPort {
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/pom.xml b/nifi/nar-bundles/framework-bundle/pom.xml
index b900c6d..b9368e8 100644
--- a/nifi/nar-bundles/framework-bundle/pom.xml
+++ b/nifi/nar-bundles/framework-bundle/pom.xml
@@ -109,6 +109,11 @@
<artifactId>web-optimistic-locking</artifactId>
<version>0.0.1-incubating-SNAPSHOT</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>site-to-site-client</artifactId>
+ <version>0.0.1-incubating-SNAPSHOT</version>
+ </dependency>
</dependencies>
</dependencyManagement>
</project>
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java b/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
new file mode 100644
index 0000000..94de86b
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
@@ -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.remote;
+
+import java.net.URI;
+import java.util.concurrent.TimeUnit;
+
+public interface RemoteDestination {
+
+ String getDescription();
+
+ String getIdentifier();
+
+ URI getTargetUri();
+
+ boolean isSecure();
+
+ long getCommunicationsTimeout(TimeUnit timeUnit);
+
+ long getYieldPeriod(TimeUnit timeUnit);
+
+ boolean isUseCompression();
+}
[2/5] incubator-nifi git commit: NIFI-282: Begin refactoring and
creating client
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
index ec169ad..cb2d76d 100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
@@ -35,8 +35,8 @@ import java.util.concurrent.atomic.AtomicReference;
import javax.net.ssl.SSLContext;
-import org.apache.nifi.cluster.NodeInformant;
import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.io.socket.SocketChannelCommunicationsSession;
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index b0d88d4..53f998e 100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -18,6 +18,7 @@ package org.apache.nifi.remote;
import java.io.DataInputStream;
import java.io.DataOutputStream;
+import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
@@ -44,18 +45,20 @@ import javax.net.ssl.SSLContext;
import javax.security.cert.CertificateExpiredException;
import javax.security.cert.CertificateNotYetValidException;
-import org.apache.nifi.cluster.ClusterNodeInformation;
-import org.apache.nifi.cluster.NodeInformation;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.connectable.Connection;
-import org.apache.nifi.controller.AbstractPort;
import org.apache.nifi.controller.ProcessScheduler;
+import org.apache.nifi.controller.exception.CommunicationsException;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.remote.client.socket.EndpointConnectionState;
+import org.apache.nifi.remote.client.socket.EndpointConnectionStatePool;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.remote.cluster.NodeInformation;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.BadRequestException;
import org.apache.nifi.remote.exception.HandshakeException;
@@ -71,35 +74,28 @@ import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.socket.SocketClientProtocol;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.scheduling.SchedulingStrategy;
-
+import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.sun.jersey.api.client.ClientHandlerException;
-public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroupPort {
+public class StandardRemoteGroupPort extends RemoteGroupPort {
public static final String USER_AGENT = "NiFi-Site-to-Site";
public static final String CONTENT_TYPE = "application/octet-stream";
public static final int GZIP_COMPRESSION_LEVEL = 1;
- public static final long PEER_REFRESH_PERIOD = 60000L;
private static final String CATEGORY = "Site to Site";
private static final Logger logger = LoggerFactory.getLogger(StandardRemoteGroupPort.class);
private final RemoteProcessGroup remoteGroup;
- private final SSLContext sslContext;
private final AtomicBoolean useCompression = new AtomicBoolean(false);
private final AtomicBoolean targetExists = new AtomicBoolean(true);
private final AtomicBoolean targetRunning = new AtomicBoolean(true);
- private final AtomicLong peerIndex = new AtomicLong(0L);
-
- private volatile List<PeerStatus> peerStatuses;
- private volatile long peerRefreshTime = 0L;
- private final ReentrantLock peerRefreshLock = new ReentrantLock();
+ private final TransferDirection transferDirection;
- private final ConcurrentMap<String, BlockingQueue<EndpointConnectionState>> endpointConnectionMap = new ConcurrentHashMap<>();
- private final ConcurrentMap<PeerStatus, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
+ private final EndpointConnectionStatePool connectionStatePool;
private final Set<CommunicationsSession> activeCommsChannels = new HashSet<>();
private final Lock interruptLock = new ReentrantLock();
@@ -113,8 +109,17 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
super(id, name, processGroup, type, scheduler);
this.remoteGroup = remoteGroup;
- this.sslContext = sslContext;
+ this.transferDirection = direction;
setScheduldingPeriod(MINIMUM_SCHEDULING_NANOS + " nanos");
+
+ final File stateDir = NiFiProperties.getInstance().getPersistentStateDirectory();
+ final File persistenceFile = new File(stateDir, remoteGroup.getIdentifier() + ".peers");
+
+ // TODO: This should really be constructed in the RemoteProcessGroup and made available to all ports in
+ // that remote process group. This prevents too many connections from being made and also protects the persistenceFile
+ // so that only a single thread will ever attempt to write to the file at once.
+ FIXME();
+ connectionStatePool = new EndpointConnectionStatePool(sslContext, remoteGroup.getEventReporter(), persistenceFile);
}
@Override
@@ -133,28 +138,15 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
@Override
public void shutdown() {
- super.shutdown();
-
- peerTimeoutExpirations.clear();
+ super.shutdown();
interruptLock.lock();
try {
this.shutdown = true;
-
- for ( final CommunicationsSession commsSession : activeCommsChannels ) {
- commsSession.interrupt();
- }
-
- for ( final BlockingQueue<EndpointConnectionState> queue : endpointConnectionMap.values() ) {
- EndpointConnectionState state;
- while ( (state = queue.poll()) != null) {
- cleanup(state.getSocketClientProtocol(), state.getPeer());
- }
- }
-
- endpointConnectionMap.clear();
} finally {
interruptLock.unlock();
}
+
+ connectionStatePool.shutdown();
}
@Override
@@ -171,31 +163,7 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
void cleanupSockets() {
- final List<EndpointConnectionState> states = new ArrayList<>();
-
- for ( final BlockingQueue<EndpointConnectionState> queue : endpointConnectionMap.values() ) {
- states.clear();
-
- EndpointConnectionState state;
- while ((state = queue.poll()) != null) {
- // If the socket has not been used in 10 seconds, shut it down.
- final long lastUsed = state.getLastTimeUsed();
- if ( lastUsed < System.currentTimeMillis() - 10000L ) {
- try {
- state.getSocketClientProtocol().shutdown(state.getPeer());
- } catch (final Exception e) {
- logger.debug("Failed to shut down {} using {} due to {}",
- new Object[] {state.getSocketClientProtocol(), state.getPeer(), e} );
- }
-
- cleanup(state.getSocketClientProtocol(), state.getPeer());
- } else {
- states.add(state);
- }
- }
-
- queue.addAll(states);
- }
+ connectionStatePool.cleanupExpiredSockets();
}
@@ -212,137 +180,46 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
}
String url = getRemoteProcessGroup().getTargetUri().toString();
- Peer peer = null;
- final PeerStatus peerStatus = getNextPeerStatus();
- if ( peerStatus == null ) {
- logger.debug("{} Unable to determine the next peer to communicate with; all peers must be penalized, so yielding context", this);
+
+ final EndpointConnectionState connectionState;
+ try {
+ connectionState = connectionStatePool.getEndpointConnectionState(url, this, transferDirection);
+ } catch (final PortNotRunningException e) {
context.yield();
+ this.targetRunning.set(false);
+ final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port is not in a valid state", this, url);
+ logger.error(message);
+ remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
return;
- }
-
- url = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort();
-
- //
- // Attempt to get a connection state that already exists for this URL.
- //
- BlockingQueue<EndpointConnectionState> connectionStateQueue = endpointConnectionMap.get(url);
- if ( connectionStateQueue == null ) {
- connectionStateQueue = new LinkedBlockingQueue<>();
- BlockingQueue<EndpointConnectionState> existingQueue = endpointConnectionMap.putIfAbsent(url, connectionStateQueue);
- if ( existingQueue != null ) {
- connectionStateQueue = existingQueue;
+ } catch (final UnknownPortException e) {
+ context.yield();
+ this.targetExists.set(false);
+ final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port no longer exists", this, url);
+ logger.error(message);
+ remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
+ return;
+ } catch (final ProtocolException | HandshakeException | IOException e) {
+ final String message = String.format("%s failed to communicate with %s due to %s", this, url, e.toString());
+ logger.error(message);
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", e);
}
+ remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
+ session.rollback();
+ return;
}
- FlowFileCodec codec = null;
- CommunicationsSession commsSession = null;
- SocketClientProtocol protocol = null;
- EndpointConnectionState connectionState;
+ if ( connectionState == null ) {
+ logger.debug("{} Unable to determine the next peer to communicate with; all peers must be penalized, so yielding context", this);
+ context.yield();
+ return;
+ }
- do {
- connectionState = connectionStateQueue.poll();
- logger.debug("{} Connection State for {} = {}", this, url, connectionState);
-
- // if we can't get an existing ConnectionState, create one
- if ( connectionState == null ) {
- protocol = new SocketClientProtocol();
- protocol.setPort(this);
-
- try {
- commsSession = establishSiteToSiteConnection(peerStatus);
- final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
- try {
- RemoteResourceFactory.initiateResourceNegotiation(protocol, dis, dos);
- } catch (final HandshakeException e) {
- try {
- commsSession.close();
- } catch (final IOException ioe) {
- final String message = String.format("%s unable to close communications session %s due to %s; resources may not be appropriately cleaned up",
- this, commsSession, ioe.toString());
- logger.error(message);
- remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
- }
- }
- } catch (final IOException e) {
- final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? url : peer, e.toString());
- logger.error(message);
- if ( logger.isDebugEnabled() ) {
- logger.error("", e);
- }
- remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
- session.rollback();
- return;
- }
-
-
- peer = new Peer(commsSession, url);
-
- // perform handshake
- try {
- protocol.handshake(peer);
-
- // handle error cases
- if ( protocol.isDestinationFull() ) {
- logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer);
- penalize(peer);
- cleanup(protocol, peer);
- return;
- } else if ( protocol.isPortInvalid() ) {
- penalize(peer);
- context.yield();
- cleanup(protocol, peer);
- this.targetRunning.set(false);
- final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port is not in a valid state", this, peer);
- logger.error(message);
- remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
- return;
- } else if ( protocol.isPortUnknown() ) {
- penalize(peer);
- context.yield();
- cleanup(protocol, peer);
- this.targetExists.set(false);
- final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port no longer exists", this, peer);
- logger.error(message);
- remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
- return;
- }
-
- // negotiate the FlowFileCodec to use
- codec = protocol.negotiateCodec(peer);
- } catch (final Exception e) {
- penalize(peer);
- cleanup(protocol, peer);
-
- final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? url : peer, e.toString());
- logger.error(message);
- if ( logger.isDebugEnabled() ) {
- logger.error("", e);
- }
- remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
- session.rollback();
- return;
- }
-
- connectionState = new EndpointConnectionState(peer, protocol, codec);
- } else {
- final long lastTimeUsed = connectionState.getLastTimeUsed();
- final long millisSinceLastUse = System.currentTimeMillis() - lastTimeUsed;
- final long timeoutMillis = remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS);
-
- if ( timeoutMillis > 0L && millisSinceLastUse >= timeoutMillis ) {
- cleanup(connectionState.getSocketClientProtocol(), connectionState.getPeer());
- connectionState = null;
- } else {
- codec = connectionState.getCodec();
- peer = connectionState.getPeer();
- commsSession = peer.getCommunicationsSession();
- protocol = connectionState.getSocketClientProtocol();
- }
- }
- } while ( connectionState == null || codec == null || commsSession == null || protocol == null );
+ FlowFileCodec codec = connectionState.getCodec();
+ SocketClientProtocol protocol = connectionState.getSocketClientProtocol();
+ final Peer peer = connectionState.getPeer();
+ url = peer.getUrl();
-
try {
interruptLock.lock();
try {
@@ -361,11 +238,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
receiveFlowFiles(peer, protocol, context, session, codec);
}
- if ( peer.isPenalized() ) {
- logger.debug("{} {} was penalized", this, peer);
- penalize(peer);
- }
-
interruptLock.lock();
try {
if ( shutdown ) {
@@ -380,12 +252,12 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
session.commit();
connectionState.setLastTimeUsed();
- connectionStateQueue.add(connectionState);
+ connectionStatePool.offer(connectionState);
} catch (final TransmissionDisabledException e) {
cleanup(protocol, peer);
session.rollback();
} catch (final Exception e) {
- penalize(peer);
+ connectionStatePool.penalize(peer, getYieldPeriod(TimeUnit.MILLISECONDS));
final String message = String.format("%s failed to communicate with %s (%s) due to %s", this, peer == null ? url : peer, protocol, e.toString());
logger.error(message);
@@ -401,34 +273,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
}
- /**
- * Updates internal state map to penalize a PeerStatus that points to the specified peer
- * @param peer
- */
- private void penalize(final Peer peer) {
- String host;
- int port;
- try {
- final URI uri = new URI(peer.getUrl());
- host = uri.getHost();
- port = uri.getPort();
- } catch (final URISyntaxException e) {
- host = peer.getHost();
- port = -1;
- }
-
- final PeerStatus status = new PeerStatus(host, port, true, 1);
- Long expiration = peerTimeoutExpirations.get(status);
- if ( expiration == null ) {
- expiration = Long.valueOf(0L);
- }
-
- final long penalizationMillis = getYieldPeriod(TimeUnit.MILLISECONDS);
- final long newExpiration = Math.max(expiration, System.currentTimeMillis() + penalizationMillis);
- peerTimeoutExpirations.put(status, Long.valueOf(newExpiration));
- }
-
-
private void cleanup(final SocketClientProtocol protocol, final Peer peer) {
if ( protocol != null && peer != null ) {
try {
@@ -457,108 +301,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
return remoteGroup.getYieldDuration();
}
- public CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException {
- final String destinationUri = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort();
-
- CommunicationsSession commsSession = null;
- try {
- if ( peerStatus.isSecure() ) {
- if ( sslContext == null ) {
- throw new IOException("Unable to communicate with " + peerStatus.getHostname() + ":" + peerStatus.getPort() + " because it requires Secure Site-to-Site communications, but this instance is not configured for secure communications");
- }
-
- final SSLSocketChannel socketChannel = new SSLSocketChannel(sslContext, peerStatus.getHostname(), peerStatus.getPort(), true);
- socketChannel.connect();
-
- commsSession = new SSLSocketChannelCommunicationsSession(socketChannel, destinationUri);
-
- try {
- commsSession.setUserDn(socketChannel.getDn());
- } catch (final CertificateNotYetValidException | CertificateExpiredException ex) {
- throw new IOException(ex);
- }
- } else {
- final SocketChannel socketChannel = SocketChannel.open(new InetSocketAddress(peerStatus.getHostname(), peerStatus.getPort()));
- commsSession = new SocketChannelCommunicationsSession(socketChannel, destinationUri);
- }
-
- commsSession.getOutput().getOutputStream().write(CommunicationsSession.MAGIC_BYTES);
-
- commsSession.setUri(destinationUri);
- } catch (final IOException ioe) {
- if ( commsSession != null ) {
- commsSession.close();
- }
-
- throw ioe;
- }
-
- return commsSession;
- }
-
- private PeerStatus getNextPeerStatus() {
- List<PeerStatus> peerList = peerStatuses;
- if ( (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD) && peerRefreshLock.tryLock() ) {
- try {
- try {
- peerList = createPeerStatusList();
- } catch (final IOException | BadRequestException | HandshakeException | UnknownPortException | PortNotRunningException | ClientHandlerException e) {
- final String message = String.format("%s Failed to update list of peers due to %s", this, e.toString());
- logger.warn(message);
- if ( logger.isDebugEnabled() ) {
- logger.warn("", e);
- }
- remoteGroup.getEventReporter().reportEvent(Severity.WARNING, CATEGORY, message);
- }
-
- this.peerStatuses = peerList;
- peerRefreshTime = System.currentTimeMillis();
- } finally {
- peerRefreshLock.unlock();
- }
- }
-
- if ( peerList == null || peerList.isEmpty() ) {
- return null;
- }
-
- PeerStatus peerStatus;
- for (int i=0; i < peerList.size(); i++) {
- final long idx = peerIndex.getAndIncrement();
- final int listIndex = (int) (idx % peerList.size());
- peerStatus = peerList.get(listIndex);
-
- if ( isPenalized(peerStatus) ) {
- logger.debug("{} {} is penalized; will not communicate with this peer", this, peerStatus);
- } else {
- return peerStatus;
- }
- }
-
- logger.debug("{} All peers appear to be penalized; returning null", this);
- return null;
- }
-
- private boolean isPenalized(final PeerStatus peerStatus) {
- final Long expirationEnd = peerTimeoutExpirations.get(peerStatus);
- return (expirationEnd == null ? false : expirationEnd > System.currentTimeMillis() );
- }
-
- private List<PeerStatus> createPeerStatusList() throws IOException, BadRequestException, HandshakeException, UnknownPortException, PortNotRunningException {
- final Set<PeerStatus> statuses = remoteGroup.getPeerStatuses();
- if ( statuses == null ) {
- return new ArrayList<>();
- }
-
- final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
- final List<NodeInformation> nodeInfos = new ArrayList<>();
- for ( final PeerStatus peerStatus : statuses ) {
- final NodeInformation nodeInfo = new NodeInformation(peerStatus.getHostname(), peerStatus.getPort(), 0, peerStatus.isSecure(), peerStatus.getFlowFileCount());
- nodeInfos.add(nodeInfo);
- }
- clusterNodeInfo.setNodeInformation(nodeInfos);
- return formulateDestinationList(clusterNodeInfo);
- }
private void transferFlowFiles(final Peer peer, final ClientProtocol protocol, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
protocol.transferFlowFiles(peer, context, session, codec);
@@ -568,70 +310,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
protocol.receiveFlowFiles(peer, context, session, codec);
}
- private List<PeerStatus> formulateDestinationList(final ClusterNodeInformation clusterNodeInfo) throws IOException {
- return formulateDestinationList(clusterNodeInfo, getConnectableType());
- }
-
- static List<PeerStatus> formulateDestinationList(final ClusterNodeInformation clusterNodeInfo, final ConnectableType connectableType) {
- final Collection<NodeInformation> nodeInfoSet = clusterNodeInfo.getNodeInformation();
- final int numDestinations = Math.max(128, nodeInfoSet.size());
- final Map<NodeInformation, Integer> entryCountMap = new HashMap<>();
-
- long totalFlowFileCount = 0L;
- for (final NodeInformation nodeInfo : nodeInfoSet) {
- totalFlowFileCount += nodeInfo.getTotalFlowFiles();
- }
-
- int totalEntries = 0;
- for (final NodeInformation nodeInfo : nodeInfoSet) {
- final int flowFileCount = nodeInfo.getTotalFlowFiles();
- // don't allow any node to get more than 80% of the data
- final double percentageOfFlowFiles = Math.min(0.8D, ((double) flowFileCount / (double) totalFlowFileCount));
- final double relativeWeighting = (connectableType == ConnectableType.REMOTE_INPUT_PORT) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles;
- final int entries = Math.max(1, (int) (numDestinations * relativeWeighting));
-
- entryCountMap.put(nodeInfo, Math.max(1, entries));
- totalEntries += entries;
- }
-
- final List<PeerStatus> destinations = new ArrayList<>(totalEntries);
- for (int i=0; i < totalEntries; i++) {
- destinations.add(null);
- }
- for ( final Map.Entry<NodeInformation, Integer> entry : entryCountMap.entrySet() ) {
- final NodeInformation nodeInfo = entry.getKey();
- final int numEntries = entry.getValue();
-
- int skipIndex = numEntries;
- for (int i=0; i < numEntries; i++) {
- int n = (skipIndex * i);
- while (true) {
- final int index = n % destinations.size();
- PeerStatus status = destinations.get(index);
- if ( status == null ) {
- status = new PeerStatus(nodeInfo.getHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure(), nodeInfo.getTotalFlowFiles());
- destinations.set(index, status);
- break;
- } else {
- n++;
- }
- }
- }
- }
-
- final StringBuilder distributionDescription = new StringBuilder();
- distributionDescription.append("New Weighted Distribution of Nodes:");
- for ( final Map.Entry<NodeInformation, Integer> entry : entryCountMap.entrySet() ) {
- final double percentage = entry.getValue() * 100D / (double) destinations.size();
- distributionDescription.append("\n").append(entry.getKey()).append(" will receive ").append(percentage).append("% of FlowFiles");
- }
- logger.info(distributionDescription.toString());
-
- // Jumble the list of destinations.
- return destinations;
- }
-
-
@Override
public boolean getTargetExists() {
return targetExists.get();
@@ -717,40 +395,6 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
}
- private static class EndpointConnectionState {
- private final Peer peer;
- private final SocketClientProtocol socketClientProtocol;
- private final FlowFileCodec codec;
- private volatile long lastUsed;
-
- private EndpointConnectionState(final Peer peer, final SocketClientProtocol socketClientProtocol, final FlowFileCodec codec) {
- this.peer = peer;
- this.socketClientProtocol = socketClientProtocol;
- this.codec = codec;
- }
-
- public FlowFileCodec getCodec() {
- return codec;
- }
-
- public SocketClientProtocol getSocketClientProtocol() {
- return socketClientProtocol;
- }
-
- public Peer getPeer() {
- return peer;
- }
-
- public void setLastTimeUsed() {
- lastUsed = System.currentTimeMillis();
- }
-
- public long getLastTimeUsed() {
- return lastUsed;
- }
- }
-
-
@Override
public SchedulingStrategy getSchedulingStrategy() {
return SchedulingStrategy.TIMER_DRIVEN;
@@ -761,4 +405,28 @@ public class StandardRemoteGroupPort extends AbstractPort implements RemoteGroup
public boolean isSideEffectFree() {
return false;
}
+
+ @Override
+ public String getDescription() {
+ return toString();
+ }
+
+ @Override
+ public long getCommunicationsTimeout(final TimeUnit timeUnit) {
+ return getRemoteProcessGroup().getCommunicationsTimeout(timeUnit);
+ }
+
+ @Override
+ public URI getTargetUri() {
+ return remoteGroup.getTargetUri();
+ }
+
+ @Override
+ public boolean isSecure() {
+ try {
+ return remoteGroup.isSecure();
+ } catch (final CommunicationsException ce) {
+ return false;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
deleted file mode 100644
index d18a4ee..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.codec;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.io.InputStreamCallback;
-import org.apache.nifi.processor.io.OutputStreamCallback;
-import org.apache.nifi.remote.StandardVersionNegotiator;
-import org.apache.nifi.remote.VersionNegotiator;
-import org.apache.nifi.remote.exception.ProtocolException;
-
-public class StandardFlowFileCodec implements FlowFileCodec {
- public static final int MAX_NUM_ATTRIBUTES = 25000;
-
- public static final String DEFAULT_FLOWFILE_PATH = "./";
-
- private final VersionNegotiator versionNegotiator;
-
- public StandardFlowFileCodec() {
- versionNegotiator = new StandardVersionNegotiator(1);
- }
-
- @Override
- public FlowFile encode(final FlowFile flowFile, final ProcessSession session, final OutputStream encodedOut) throws IOException {
- final DataOutputStream out = new DataOutputStream(encodedOut);
-
- final Map<String, String> attributes = flowFile.getAttributes();
- out.writeInt(attributes.size());
- for ( final Map.Entry<String, String> entry : attributes.entrySet() ) {
- writeString(entry.getKey(), out);
- writeString(entry.getValue(), out);
- }
-
- out.writeLong(flowFile.getSize());
-
- session.read(flowFile, new InputStreamCallback() {
- @Override
- public void process(final InputStream in) throws IOException {
- final byte[] buffer = new byte[8192];
- int len;
- while ( (len = in.read(buffer)) > 0 ) {
- encodedOut.write(buffer, 0, len);
- }
-
- encodedOut.flush();
- }
- });
-
- return flowFile;
- }
-
-
- @Override
- public FlowFile decode(final InputStream stream, final ProcessSession session) throws IOException, ProtocolException {
- final DataInputStream in = new DataInputStream(stream);
-
- final int numAttributes;
- try {
- numAttributes = in.readInt();
- } catch (final EOFException e) {
- // we're out of data.
- return null;
- }
-
- // This is here because if the stream is not properly formed, we could get up to Integer.MAX_VALUE attributes, which will
- // generally result in an OutOfMemoryError.
- if ( numAttributes > MAX_NUM_ATTRIBUTES ) {
- throw new ProtocolException("FlowFile exceeds maximum number of attributes with a total of " + numAttributes);
- }
-
- try {
- final Map<String, String> attributes = new HashMap<>(numAttributes);
- for (int i=0; i < numAttributes; i++) {
- final String attrName = readString(in);
- final String attrValue = readString(in);
- attributes.put(attrName, attrValue);
- }
-
- final long numBytes = in.readLong();
-
- FlowFile flowFile = session.create();
- flowFile = session.putAllAttributes(flowFile, attributes);
- flowFile = session.write(flowFile, new OutputStreamCallback() {
- @Override
- public void process(final OutputStream out) throws IOException {
- int len;
- long size = 0;
- final byte[] buffer = new byte[8192];
-
- while ( size < numBytes && (len = in.read(buffer, 0, (int) Math.min(buffer.length, numBytes - size))) > 0 ) {
- out.write(buffer, 0, len);
- size += len;
- }
-
- if ( size != numBytes ) {
- throw new EOFException("Expected " + numBytes + " bytes but received only " + size);
- }
- }
- });
-
- return flowFile;
- } catch (final EOFException e) {
- session.rollback();
-
- // we throw the general IOException here because we did not expect to hit EOFException
- throw e;
- }
- }
-
- private void writeString(final String val, final DataOutputStream out) throws IOException {
- final byte[] bytes = val.getBytes("UTF-8");
- out.writeInt(bytes.length);
- out.write(bytes);
- }
-
-
- private String readString(final DataInputStream in) throws IOException {
- final int numBytes = in.readInt();
- final byte[] bytes = new byte[numBytes];
- StreamUtils.fillBuffer(in, bytes, true);
- return new String(bytes, "UTF-8");
- }
-
- @Override
- public List<Integer> getSupportedVersions() {
- return versionNegotiator.getSupportedVersions();
- }
-
- @Override
- public VersionNegotiator getVersionNegotiator() {
- return versionNegotiator;
- }
-
- @Override
- public String toString() {
- return "Standard FlowFile Codec, Version " + versionNegotiator.getVersion();
- }
-
- @Override
- public String getResourceName() {
- return "StandardFlowFileCodec";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java
deleted file mode 100644
index 926809c..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-import org.apache.nifi.remote.codec.FlowFileCodec;
-
-public class UnsupportedCodecException extends RuntimeException {
- private static final long serialVersionUID = 198234789237L;
-
- public UnsupportedCodecException(final String codecName) {
- super("Codec " + codecName + " is not supported");
- }
-
- public UnsupportedCodecException(final FlowFileCodec codec, final int version) {
- super("Codec " + codec.getClass().getName() + " does not support Version " + version);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
deleted file mode 100644
index 0822b6a..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.io.socket;
-
-import java.io.IOException;
-import java.nio.channels.SocketChannel;
-
-import org.apache.nifi.remote.AbstractCommunicationsSession;
-
-public class SocketChannelCommunicationsSession extends AbstractCommunicationsSession {
- private final SocketChannel channel;
- private final SocketChannelInput request;
- private final SocketChannelOutput response;
- private int timeout = 30000;
-
- public SocketChannelCommunicationsSession(final SocketChannel socketChannel, final String uri) throws IOException {
- super(uri);
- request = new SocketChannelInput(socketChannel);
- response = new SocketChannelOutput(socketChannel);
- channel = socketChannel;
- socketChannel.configureBlocking(false);
- }
-
- @Override
- public boolean isClosed() {
- return !channel.isConnected();
- }
-
- @Override
- public SocketChannelInput getInput() {
- return request;
- }
-
- @Override
- public SocketChannelOutput getOutput() {
- return response;
- }
-
- @Override
- public void setTimeout(final int millis) throws IOException {
- request.setTimeout(millis);
- response.setTimeout(millis);
- this.timeout = millis;
- }
-
- @Override
- public int getTimeout() throws IOException {
- return timeout;
- }
-
- @Override
- public void close() throws IOException {
- channel.close();
- }
-
- @Override
- public boolean isDataAvailable() {
- return request.isDataAvailable();
- }
-
- @Override
- public long getBytesWritten() {
- return response.getBytesWritten();
- }
-
- @Override
- public long getBytesRead() {
- return request.getBytesRead();
- }
-
- @Override
- public void interrupt() {
- request.interrupt();
- response.interrupt();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
deleted file mode 100644
index 9e451fd..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.io.socket;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.channels.SocketChannel;
-
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.stream.io.ByteCountingInputStream;
-import org.apache.nifi.remote.io.InterruptableInputStream;
-import org.apache.nifi.remote.protocol.CommunicationsInput;
-
-public class SocketChannelInput implements CommunicationsInput {
- private final SocketChannelInputStream socketIn;
- private final ByteCountingInputStream countingIn;
- private final InputStream bufferedIn;
- private final InterruptableInputStream interruptableIn;
-
- public SocketChannelInput(final SocketChannel socketChannel) throws IOException {
- this.socketIn = new SocketChannelInputStream(socketChannel);
- countingIn = new ByteCountingInputStream(socketIn);
- bufferedIn = new BufferedInputStream(countingIn);
- interruptableIn = new InterruptableInputStream(bufferedIn);
- }
-
- @Override
- public InputStream getInputStream() throws IOException {
- return interruptableIn;
- }
-
- public void setTimeout(final int millis) {
- socketIn.setTimeout(millis);
- }
-
- public boolean isDataAvailable() {
- try {
- return interruptableIn.available() > 0;
- } catch (final Exception e) {
- return false;
- }
- }
-
- @Override
- public long getBytesRead() {
- return countingIn.getBytesRead();
- }
-
- public void interrupt() {
- interruptableIn.interrupt();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
deleted file mode 100644
index 26c0164..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.io.socket;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.channels.SocketChannel;
-
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.ByteCountingOutputStream;
-import org.apache.nifi.remote.io.InterruptableOutputStream;
-import org.apache.nifi.remote.protocol.CommunicationsOutput;
-
-public class SocketChannelOutput implements CommunicationsOutput {
- private final SocketChannelOutputStream socketOutStream;
- private final ByteCountingOutputStream countingOut;
- private final OutputStream bufferedOut;
- private final InterruptableOutputStream interruptableOut;
-
- public SocketChannelOutput(final SocketChannel socketChannel) throws IOException {
- socketOutStream = new SocketChannelOutputStream(socketChannel);
- countingOut = new ByteCountingOutputStream(socketOutStream);
- bufferedOut = new BufferedOutputStream(countingOut);
- interruptableOut = new InterruptableOutputStream(bufferedOut);
- }
-
- @Override
- public OutputStream getOutputStream() throws IOException {
- return interruptableOut;
- }
-
- public void setTimeout(final int timeout) {
- socketOutStream.setTimeout(timeout);
- }
-
- @Override
- public long getBytesWritten() {
- return countingOut.getBytesWritten();
- }
-
- public void interrupt() {
- interruptableOut.interrupt();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
deleted file mode 100644
index dca1d84..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.io.socket.ssl;
-
-import java.io.IOException;
-
-import org.apache.nifi.remote.AbstractCommunicationsSession;
-
-public class SSLSocketChannelCommunicationsSession extends AbstractCommunicationsSession {
- private final SSLSocketChannel channel;
- private final SSLSocketChannelInput request;
- private final SSLSocketChannelOutput response;
-
- public SSLSocketChannelCommunicationsSession(final SSLSocketChannel channel, final String uri) {
- super(uri);
- request = new SSLSocketChannelInput(channel);
- response = new SSLSocketChannelOutput(channel);
- this.channel = channel;
- }
-
- @Override
- public SSLSocketChannelInput getInput() {
- return request;
- }
-
- @Override
- public SSLSocketChannelOutput getOutput() {
- return response;
- }
-
- @Override
- public void setTimeout(final int millis) throws IOException {
- channel.setTimeout(millis);
- }
-
- @Override
- public int getTimeout() throws IOException {
- return channel.getTimeout();
- }
-
- @Override
- public void close() throws IOException {
- channel.close();
- }
-
- @Override
- public boolean isClosed() {
- return channel.isClosed();
- }
-
- @Override
- public boolean isDataAvailable() {
- try {
- return request.isDataAvailable();
- } catch (final Exception e) {
- return false;
- }
- }
-
- @Override
- public long getBytesWritten() {
- return response.getBytesWritten();
- }
-
- @Override
- public long getBytesRead() {
- return request.getBytesRead();
- }
-
- @Override
- public void interrupt() {
- channel.interrupt();
- }
-
- @Override
- public String toString() {
- return super.toString() + "[SSLSocketChannel=" + channel + "]";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
deleted file mode 100644
index 60ef33f..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.io.socket.ssl;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.stream.io.ByteCountingInputStream;
-import org.apache.nifi.remote.protocol.CommunicationsInput;
-
-public class SSLSocketChannelInput implements CommunicationsInput {
- private final SSLSocketChannelInputStream in;
- private final ByteCountingInputStream countingIn;
- private final InputStream bufferedIn;
-
- public SSLSocketChannelInput(final SSLSocketChannel socketChannel) {
- in = new SSLSocketChannelInputStream(socketChannel);
- countingIn = new ByteCountingInputStream(in);
- this.bufferedIn = new BufferedInputStream(countingIn);
- }
-
- @Override
- public InputStream getInputStream() throws IOException {
- return bufferedIn;
- }
-
- public boolean isDataAvailable() throws IOException {
- return bufferedIn.available() > 0;
- }
-
- @Override
- public long getBytesRead() {
- return countingIn.getBytesRead();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
deleted file mode 100644
index dc3d68f..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.io.socket.ssl;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.ByteCountingOutputStream;
-import org.apache.nifi.remote.protocol.CommunicationsOutput;
-
-public class SSLSocketChannelOutput implements CommunicationsOutput {
- private final OutputStream out;
- private final ByteCountingOutputStream countingOut;
-
- public SSLSocketChannelOutput(final SSLSocketChannel channel) {
- countingOut = new ByteCountingOutputStream(new SSLSocketChannelOutputStream(channel));
- out = new BufferedOutputStream(countingOut);
- }
-
- @Override
- public OutputStream getOutputStream() throws IOException {
- return out;
- }
-
- @Override
- public long getBytesWritten() {
- return countingOut.getBytesWritten();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java
index a526f4c..391d52b 100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java
@@ -21,9 +21,6 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Collection;
-import org.apache.nifi.cluster.ClusterNodeInformation;
-import org.apache.nifi.cluster.NodeInformant;
-import org.apache.nifi.cluster.NodeInformation;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@@ -31,12 +28,14 @@ import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.StandardVersionNegotiator;
import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.remote.cluster.NodeInformant;
+import org.apache.nifi.remote.cluster.NodeInformation;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.protocol.CommunicationsSession;
import org.apache.nifi.remote.protocol.RequestType;
import org.apache.nifi.remote.protocol.ServerProtocol;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
deleted file mode 100644
index c4519cd..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol.socket;
-
-public enum HandshakeProperty {
- GZIP,
- PORT_IDENTIFIER,
- REQUEST_EXPIRATION_MILLIS;
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
deleted file mode 100644
index eae1940..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol.socket;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-
-import org.apache.nifi.remote.exception.ProtocolException;
-
-public class Response {
- private final ResponseCode code;
- private final String message;
-
- private Response(final ResponseCode code, final String explanation) {
- this.code = code;
- this.message = explanation;
- }
-
- public ResponseCode getCode() {
- return code;
- }
-
- public String getMessage() {
- return message;
- }
-
- public static Response read(final DataInputStream in) throws IOException, ProtocolException {
- final ResponseCode code = ResponseCode.readCode(in);
- final String message = code.containsMessage() ? in.readUTF() : null;
- return new Response(code, message);
- }
-
- @Override
- public String toString() {
- return code + ": " + message;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
deleted file mode 100644
index 0e588cd..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol.socket;
-
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.nifi.remote.exception.ProtocolException;
-
-
-public enum ResponseCode {
- RESERVED(0, "Reserved for Future Use", false), // This will likely be used if we ever need to expand the length of
- // ResponseCode, so that we can indicate a 0 followed by some other bytes
-
- // handshaking properties
- PROPERTIES_OK(1, "Properties OK", false),
- UNKNOWN_PROPERTY_NAME(230, "Unknown Property Name", true),
- ILLEGAL_PROPERTY_VALUE(231, "Illegal Property Value", true),
- MISSING_PROPERTY(232, "Missing Property", true),
-
- // transaction indicators
- CONTINUE_TRANSACTION(10, "Continue Transaction", false),
- FINISH_TRANSACTION(11, "Finish Transaction", false),
- CONFIRM_TRANSACTION(12, "Confirm Transaction", true), // "Explanation" of this code is the checksum
- TRANSACTION_FINISHED(13, "Transaction Finished", false),
- TRANSACTION_FINISHED_BUT_DESTINATION_FULL(14, "Transaction Finished But Destination is Full", false),
- BAD_CHECKSUM(19, "Bad Checksum", false),
-
- // data availability indicators
- MORE_DATA(20, "More Data Exists", false),
- NO_MORE_DATA(21, "No More Data Exists", false),
-
- // port state indicators
- UNKNOWN_PORT(200, "Unknown Port", false),
- PORT_NOT_IN_VALID_STATE(201, "Port Not in a Valid State", true),
- PORTS_DESTINATION_FULL(202, "Port's Destination is Full", false),
-
- // authorization
- UNAUTHORIZED(240, "User Not Authorized", true),
-
- // error indicators
- ABORT(250, "Abort", true),
- UNRECOGNIZED_RESPONSE_CODE(254, "Unrecognized Response Code", false),
- END_OF_STREAM(255, "End of Stream", false);
-
- private static final ResponseCode[] codeArray = new ResponseCode[256];
-
- static {
- for ( final ResponseCode responseCode : ResponseCode.values() ) {
- codeArray[responseCode.getCode()] = responseCode;
- }
- }
-
- private static final byte CODE_SEQUENCE_VALUE_1 = (byte) 'R';
- private static final byte CODE_SEQUENCE_VALUE_2 = (byte) 'C';
- private final int code;
- private final byte[] codeSequence;
- private final String description;
- private final boolean containsMessage;
-
- private ResponseCode(final int code, final String description, final boolean containsMessage) {
- this.codeSequence = new byte[] {CODE_SEQUENCE_VALUE_1, CODE_SEQUENCE_VALUE_2, (byte) code};
- this.code = code;
- this.description = description;
- this.containsMessage = containsMessage;
- }
-
- public int getCode() {
- return code;
- }
-
- public byte[] getCodeSequence() {
- return codeSequence;
- }
-
- @Override
- public String toString() {
- return description;
- }
-
- public boolean containsMessage() {
- return containsMessage;
- }
-
- public void writeResponse(final DataOutputStream out) throws IOException {
- if ( containsMessage() ) {
- throw new IllegalArgumentException("ResponseCode " + code + " expects an explanation");
- }
-
- out.write(getCodeSequence());
- out.flush();
- }
-
- public void writeResponse(final DataOutputStream out, final String explanation) throws IOException {
- if ( !containsMessage() ) {
- throw new IllegalArgumentException("ResponseCode " + code + " does not expect an explanation");
- }
-
- out.write(getCodeSequence());
- out.writeUTF(explanation);
- out.flush();
- }
-
- static ResponseCode readCode(final InputStream in) throws IOException, ProtocolException {
- final int byte1 = in.read();
- if ( byte1 < 0 ) {
- throw new EOFException();
- } else if ( byte1 != CODE_SEQUENCE_VALUE_1 ) {
- throw new ProtocolException("Expected to receive ResponseCode, but the stream did not have a ResponseCode");
- }
-
- final int byte2 = in.read();
- if ( byte2 < 0 ) {
- throw new EOFException();
- } else if ( byte2 != CODE_SEQUENCE_VALUE_2 ) {
- throw new ProtocolException("Expected to receive ResponseCode, but the stream did not have a ResponseCode");
- }
-
- final int byte3 = in.read();
- if ( byte3 < 0 ) {
- throw new EOFException();
- }
-
- final ResponseCode responseCode = codeArray[byte3];
- if (responseCode == null) {
- throw new ProtocolException("Received Response Code of " + byte3 + " but do not recognize this code");
- }
- return responseCode;
- }
-
- public static ResponseCode fromSequence(final byte[] value) {
- final int code = value[3] & 0xFF;
- final ResponseCode responseCode = codeArray[code];
- return (responseCode == null) ? UNRECOGNIZED_RESPONSE_CODE : responseCode;
- }
-}
\ No newline at end of file
[3/5] incubator-nifi git commit: NIFI-282: Begin refactoring and
creating client
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java
new file mode 100644
index 0000000..6dab77b
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.util;
+
+import java.util.Set;
+
+import org.apache.nifi.remote.PeerStatus;
+
+public class PeerStatusCache {
+ private final Set<PeerStatus> statuses;
+ private final long timestamp;
+
+ public PeerStatusCache(final Set<PeerStatus> statuses) {
+ this(statuses, System.currentTimeMillis());
+ }
+
+ public PeerStatusCache(final Set<PeerStatus> statuses, final long timestamp) {
+ this.statuses = statuses;
+ this.timestamp = timestamp;
+ }
+
+ public Set<PeerStatus> getStatuses() {
+ return statuses;
+ }
+
+ public long getTimestamp() {
+ return timestamp;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java b/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
index 0fcac8c..00a7687 100644
--- a/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
+++ b/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
@@ -23,7 +23,6 @@ import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException;
import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException;
-import org.apache.nifi.cluster.NodeInformant;
import org.apache.nifi.cluster.event.Event;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.node.Node.Status;
@@ -33,6 +32,7 @@ import org.apache.nifi.cluster.protocol.Heartbeat;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.diagnostics.SystemDiagnostics;
+import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.reporting.BulletinRepository;
import java.util.List;
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
index 511bb7d..ea523b0 100644
--- a/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
+++ b/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
@@ -65,9 +65,7 @@ import javax.xml.validation.Validator;
import org.apache.nifi.admin.service.AuditService;
import org.apache.nifi.cluster.BulletinsPayload;
-import org.apache.nifi.cluster.ClusterNodeInformation;
import org.apache.nifi.cluster.HeartbeatPayload;
-import org.apache.nifi.cluster.NodeInformation;
import org.apache.nifi.cluster.context.ClusterContext;
import org.apache.nifi.cluster.context.ClusterContextImpl;
import org.apache.nifi.cluster.event.Event;
@@ -155,6 +153,8 @@ import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.remote.RemoteResourceManager;
import org.apache.nifi.remote.RemoteSiteListener;
import org.apache.nifi.remote.SocketRemoteSiteListener;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.remote.cluster.NodeInformation;
import org.apache.nifi.remote.protocol.socket.ClusterManagerServerProtocol;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
@@ -191,7 +191,6 @@ import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
import org.apache.nifi.web.util.WebUtils;
-
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/pom.xml b/nifi/nar-bundles/framework-bundle/framework/core-api/pom.xml
index f8d8e13..643121e 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/pom.xml
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/pom.xml
@@ -34,6 +34,11 @@
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
+ <artifactId>site-to-site-client</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
<artifactId>nifi-runtime</artifactId>
</dependency>
<dependency>
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
deleted file mode 100644
index 0092f7a..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster;
-
-public class AdaptedNodeInformation {
-
- private String hostname;
- private Integer siteToSitePort;
- private int apiPort;
- private boolean isSiteToSiteSecure;
- private int totalFlowFiles;
-
- public String getHostname() {
- return hostname;
- }
-
- public void setHostname(String hostname) {
- this.hostname = hostname;
- }
-
- public Integer getSiteToSitePort() {
- return siteToSitePort;
- }
-
- public void setSiteToSitePort(Integer siteToSitePort) {
- this.siteToSitePort = siteToSitePort;
- }
-
- public int getApiPort() {
- return apiPort;
- }
-
- public void setApiPort(int apiPort) {
- this.apiPort = apiPort;
- }
-
- public boolean isSiteToSiteSecure() {
- return isSiteToSiteSecure;
- }
-
- public void setSiteToSiteSecure(boolean isSiteToSiteSecure) {
- this.isSiteToSiteSecure = isSiteToSiteSecure;
- }
-
- public int getTotalFlowFiles() {
- return totalFlowFiles;
- }
-
- public void setTotalFlowFiles(int totalFlowFiles) {
- this.totalFlowFiles = totalFlowFiles;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
deleted file mode 100644
index 5751c32..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collection;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.annotation.XmlRootElement;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-@XmlRootElement
-public class ClusterNodeInformation {
-
- private Collection<NodeInformation> nodeInfo;
-
- private static final JAXBContext JAXB_CONTEXT;
-
- static {
- try {
- JAXB_CONTEXT = JAXBContext.newInstance(ClusterNodeInformation.class);
- } catch (JAXBException e) {
- throw new RuntimeException("Unable to create JAXBContext.", e);
- }
- }
-
- public ClusterNodeInformation() {
- this.nodeInfo = null;
- }
-
- public void setNodeInformation(final Collection<NodeInformation> nodeInfo) {
- this.nodeInfo = nodeInfo;
- }
-
- @XmlJavaTypeAdapter(NodeInformationAdapter.class)
- public Collection<NodeInformation> getNodeInformation() {
- return nodeInfo;
- }
-
- public void marshal(final OutputStream os) throws JAXBException {
- final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
- marshaller.marshal(this, os);
- }
-
- public static ClusterNodeInformation unmarshal(final InputStream is) throws JAXBException {
- final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
- return (ClusterNodeInformation) unmarshaller.unmarshal(is);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
deleted file mode 100644
index 987ff65..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster;
-
-public interface NodeInformant {
-
- ClusterNodeInformation getNodeInformation();
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
deleted file mode 100644
index 848eb7e..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster;
-
-public class NodeInformation {
-
- private final String hostname;
- private final Integer siteToSitePort;
- private final int apiPort;
- private final boolean isSiteToSiteSecure;
- private final int totalFlowFiles;
-
- public NodeInformation(final String hostname, final Integer siteToSitePort, final int apiPort,
- final boolean isSiteToSiteSecure, final int totalFlowFiles) {
- this.hostname = hostname;
- this.siteToSitePort = siteToSitePort;
- this.apiPort = apiPort;
- this.isSiteToSiteSecure = isSiteToSiteSecure;
- this.totalFlowFiles = totalFlowFiles;
- }
-
- public String getHostname() {
- return hostname;
- }
-
- public int getAPIPort() {
- return apiPort;
- }
-
- public Integer getSiteToSitePort() {
- return siteToSitePort;
- }
-
- public boolean isSiteToSiteSecure() {
- return isSiteToSiteSecure;
- }
-
- public int getTotalFlowFiles() {
- return totalFlowFiles;
- }
-
- @Override
- public boolean equals(final Object obj) {
- if (obj == this) {
- return true;
- }
- if (obj == null) {
- return false;
- }
- if (!(obj instanceof NodeInformation)) {
- return false;
- }
-
- final NodeInformation other = (NodeInformation) obj;
- if (!hostname.equals(other.hostname)) {
- return false;
- }
- if (siteToSitePort == null && other.siteToSitePort != null) {
- return false;
- }
- if (siteToSitePort != null && other.siteToSitePort == null) {
- return false;
- } else if (siteToSitePort != null && siteToSitePort.intValue() != other.siteToSitePort.intValue()) {
- return false;
- }
- if (apiPort != other.apiPort) {
- return false;
- }
- if (isSiteToSiteSecure != other.isSiteToSiteSecure) {
- return false;
- }
- return true;
- }
-
- @Override
- public int hashCode() {
- return 83832 + hostname.hashCode() + (siteToSitePort == null ? 8 : siteToSitePort.hashCode()) + apiPort + (isSiteToSiteSecure ? 3829 : 0);
- }
-
- @Override
- public String toString() {
- return "Node[" + hostname + ":" + apiPort + "]";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
deleted file mode 100644
index 630631f..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.cluster;
-
-import javax.xml.bind.annotation.adapters.XmlAdapter;
-
-public class NodeInformationAdapter extends XmlAdapter<AdaptedNodeInformation, NodeInformation> {
-
- @Override
- public NodeInformation unmarshal(final AdaptedNodeInformation adapted) throws Exception {
- return new NodeInformation(adapted.getHostname(), adapted.getSiteToSitePort(), adapted.getApiPort(), adapted.isSiteToSiteSecure(), adapted.getTotalFlowFiles());
- }
-
- @Override
- public AdaptedNodeInformation marshal(final NodeInformation nodeInformation) throws Exception {
- final AdaptedNodeInformation adapted = new AdaptedNodeInformation();
- adapted.setHostname(nodeInformation.getHostname());
- adapted.setSiteToSitePort(nodeInformation.getSiteToSitePort());
- adapted.setApiPort(nodeInformation.getAPIPort());
- adapted.setSiteToSiteSecure(nodeInformation.isSiteToSiteSecure());
- adapted.setTotalFlowFiles(nodeInformation.getTotalFlowFiles());
- return adapted;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
index e0cca64..2e35422 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
@@ -27,7 +27,6 @@ import org.apache.nifi.controller.exception.CommunicationsException;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.PeerStatus;
import org.apache.nifi.remote.RemoteGroupPort;
-import org.apache.nifi.remote.protocol.CommunicationsSession;
public interface RemoteProcessGroup {
@@ -211,11 +210,6 @@ public interface RemoteProcessGroup {
*/
void removeNonExistentPort(final RemoteGroupPort port);
- /**
- *
- * @return @throws IOException
- */
- CommunicationsSession establishSiteToSiteConnection() throws IOException;
/**
* Called whenever RemoteProcessGroup is removed from the flow, so that any
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java
deleted file mode 100644
index 2422fe1..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/Peer.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote;
-
-import java.io.IOException;
-import java.net.URI;
-
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-
-public class Peer {
-
- private final CommunicationsSession commsSession;
- private final String url;
- private final String host;
- private long penalizationExpiration = 0L;
- private boolean closed = false;
-
- public Peer(final CommunicationsSession commsSession, final String url) {
- this.commsSession = commsSession;
- this.url = url;
-
- try {
- this.host = new URI(url).getHost();
- } catch (final Exception e) {
- throw new IllegalArgumentException("Invalid URL: " + url);
- }
- }
-
- public String getUrl() {
- return url;
- }
-
- public CommunicationsSession getCommunicationsSession() {
- return commsSession;
- }
-
- public void close() throws IOException {
- this.closed = true;
-
- // TODO: Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer
- commsSession.close();
- }
-
- public void penalize(final long millis) {
- penalizationExpiration = Math.max(penalizationExpiration, System.currentTimeMillis() + millis);
- }
-
- public boolean isPenalized() {
- return penalizationExpiration > System.currentTimeMillis();
- }
-
- public boolean isClosed() {
- return closed;
- }
-
- public String getHost() {
- return host;
- }
-
- @Override
- public int hashCode() {
- return 8320 + url.hashCode();
- }
-
- @Override
- public boolean equals(final Object obj) {
- if (obj == null) {
- return false;
- }
- if (obj == this) {
- return true;
- }
- if (!(obj instanceof Peer)) {
- return false;
- }
-
- final Peer other = (Peer) obj;
- return this.url.equals(other.url);
- }
-
- @Override
- public String toString() {
- final StringBuilder sb = new StringBuilder();
- sb.append("Peer[url=").append(url);
- if (closed) {
- sb.append(",CLOSED");
- } else if (isPenalized()) {
- sb.append(",PENALIZED");
- }
- sb.append("]");
- return sb.toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
deleted file mode 100644
index d1cb076..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote;
-
-public class PeerStatus {
-
- private final String hostname;
- private final int port;
- private final boolean secure;
- private final int numFlowFiles;
-
- public PeerStatus(final String hostname, final int port, final boolean secure, final int numFlowFiles) {
- this.hostname = hostname;
- this.port = port;
- this.secure = secure;
- this.numFlowFiles = numFlowFiles;
- }
-
- public String getHostname() {
- return hostname;
- }
-
- public int getPort() {
- return port;
- }
-
- public boolean isSecure() {
- return secure;
- }
-
- public int getFlowFileCount() {
- return numFlowFiles;
- }
-
- @Override
- public String toString() {
- return "PeerStatus[hostname=" + hostname + ",port=" + port + ",secure=" + secure + ",flowFileCount=" + numFlowFiles + "]";
- }
-
- @Override
- public int hashCode() {
- return 9824372 + hostname.hashCode() + port;
- }
-
- @Override
- public boolean equals(final Object obj) {
- if (obj == null) {
- return false;
- }
-
- if (!(obj instanceof PeerStatus)) {
- return false;
- }
-
- final PeerStatus other = (PeerStatus) obj;
- return port == other.port && hostname.equals(other.hostname);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
deleted file mode 100644
index 8f2603a..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote;
-
-public interface PortAuthorizationResult {
-
- boolean isAuthorized();
-
- String getExplanation();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
deleted file mode 100644
index 12a3d33..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote;
-
-/**
- *
- */
-public enum RemoteAuthorizationState {
-
- UNKNOWN,
- UNAUTHORIZED,
- AUTHORIZED;
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
index d4ad374..f08277c 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
@@ -16,20 +16,28 @@
*/
package org.apache.nifi.remote;
+import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.connectable.Port;
+import org.apache.nifi.controller.AbstractPort;
+import org.apache.nifi.controller.ProcessScheduler;
+import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
-public interface RemoteGroupPort extends Port {
+public abstract class RemoteGroupPort extends AbstractPort implements Port, RemoteDestination {
- RemoteProcessGroup getRemoteProcessGroup();
+ public RemoteGroupPort(String id, String name, ProcessGroup processGroup, ConnectableType type, ProcessScheduler scheduler) {
+ super(id, name, processGroup, type, scheduler);
+ }
- TransferDirection getTransferDirection();
+ public abstract RemoteProcessGroup getRemoteProcessGroup();
- boolean isUseCompression();
+ public abstract TransferDirection getTransferDirection();
- void setUseCompression(boolean useCompression);
+ public abstract boolean isUseCompression();
- boolean getTargetExists();
+ public abstract void setUseCompression(boolean useCompression);
- boolean isTargetRunning();
+ public abstract boolean getTargetExists();
+
+ public abstract boolean isTargetRunning();
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
deleted file mode 100644
index 56432d5..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote;
-
-public enum TransferDirection {
-
- SEND,
- RECEIVE;
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
deleted file mode 100644
index bfccd98..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote;
-
-public interface VersionedRemoteResource {
-
- VersionNegotiator getVersionNegotiator();
-
- String getResourceName();
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
deleted file mode 100644
index b4206b3..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.codec;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.List;
-
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.remote.VersionedRemoteResource;
-import org.apache.nifi.remote.exception.ProtocolException;
-import org.apache.nifi.remote.exception.TransmissionDisabledException;
-
-/**
- * <p>
- * Provides a mechanism for encoding and decoding FlowFiles as streams so that
- * they can be transferred remotely.
- * </p>
- */
-public interface FlowFileCodec extends VersionedRemoteResource {
-
- /**
- * Returns a List of all versions that this codec is able to support, in the
- * order that they are preferred by the codec
- *
- * @return
- */
- public List<Integer> getSupportedVersions();
-
- /**
- * Encodes a FlowFile and its content as a single stream of data and writes
- * that stream to the output. If checksum is not null, it will be calculated
- * as the stream is read
- *
- * @param flowFile the FlowFile to encode
- * @param session a session that can be used to transactionally create and
- * transfer flow files
- * @param outStream the stream to write the data to
- *
- * @return the updated FlowFile
- *
- * @throws IOException
- */
- FlowFile encode(FlowFile flowFile, ProcessSession session, OutputStream outStream) throws IOException, TransmissionDisabledException;
-
- /**
- * Decodes the contents of the InputStream, interpreting the data to
- * determine the next FlowFile's attributes and content, as well as their
- * destinations. If not null, checksum will be used to calculate the
- * checksum as the data is read.
- *
- * @param stream an InputStream containing FlowFiles' contents, attributes,
- * and destinations
- * @param session
- *
- * @return the FlowFile that was created, or <code>null</code> if the stream
- * was out of data
- *
- * @throws IOException
- * @throws ProtocolException if the input is malformed
- */
- FlowFile decode(InputStream stream, ProcessSession session) throws IOException, ProtocolException, TransmissionDisabledException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
deleted file mode 100644
index f6c2f4f..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-public class BadRequestException extends Exception {
-
- private static final long serialVersionUID = -8034602852256106560L;
-
- public BadRequestException(final String message) {
- super(message);
- }
-
- public BadRequestException(final Throwable cause) {
- super(cause);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
deleted file mode 100644
index b61fc65..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-public class HandshakeException extends Exception {
-
- private static final long serialVersionUID = 178192341908726L;
-
- public HandshakeException(final String message) {
- super(message);
- }
-
- public HandshakeException(final Throwable cause) {
- super(cause);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
deleted file mode 100644
index 24ff3a5..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-public class NotAuthorizedException extends Exception {
-
- private static final long serialVersionUID = 2952623568114035498L;
-
- public NotAuthorizedException(final String message) {
- super(message);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
deleted file mode 100644
index af0f467..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-public class PortNotRunningException extends Exception {
-
- private static final long serialVersionUID = -2790940982005516375L;
-
- public PortNotRunningException(final String message) {
- super(message);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
deleted file mode 100644
index 0f50b98..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-public class ProtocolException extends Exception {
-
- private static final long serialVersionUID = 5763900324505818495L;
-
- public ProtocolException(final String message, final Throwable cause) {
- super(message, cause);
- }
-
- public ProtocolException(final String message) {
- super(message);
- }
-
- public ProtocolException(final Throwable cause) {
- super(cause);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
deleted file mode 100644
index dd675b3..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-/**
- * Used to indicate that by the time the request was serviced, it had already
- * expired
- */
-public class RequestExpiredException extends Exception {
-
- private static final long serialVersionUID = -7037025330562827852L;
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
deleted file mode 100644
index e6a0fe7..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.exception;
-
-public class UnknownPortException extends Exception {
-
- private static final long serialVersionUID = -2790940982005516375L;
-
- public UnknownPortException(final String message) {
- super(message);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
deleted file mode 100644
index 32274eb..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol;
-
-import java.io.IOException;
-import java.util.Set;
-
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.remote.Peer;
-import org.apache.nifi.remote.PeerStatus;
-import org.apache.nifi.remote.VersionedRemoteResource;
-import org.apache.nifi.remote.codec.FlowFileCodec;
-import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.PortNotRunningException;
-import org.apache.nifi.remote.exception.ProtocolException;
-import org.apache.nifi.remote.exception.UnknownPortException;
-
-public interface ClientProtocol extends VersionedRemoteResource {
-
- void handshake(Peer peer) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException;
-
- Set<PeerStatus> getPeerStatuses(Peer currentPeer) throws IOException, ProtocolException;
-
- FlowFileCodec negotiateCodec(Peer peer) throws IOException, ProtocolException;
-
- void receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
-
- void transferFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
-
- void shutdown(Peer peer) throws IOException, ProtocolException;
-
- boolean isReadyForFileTransfer();
-
- /**
- * returns <code>true</code> if remote instance indicates that the port is
- * invalid
- *
- * @return
- * @throws IllegalStateException if a handshake has not successfully
- * completed
- */
- boolean isPortInvalid() throws IllegalStateException;
-
- /**
- * returns <code>true</code> if remote instance indicates that the port is
- * unknown
- *
- * @return
- * @throws IllegalStateException if a handshake has not successfully
- * completed
- */
- boolean isPortUnknown();
-
- /**
- * returns <code>true</code> if remote instance indicates that the port's
- * destination is full
- *
- * @return
- * @throws IllegalStateException if a handshake has not successfully
- * completed
- */
- boolean isDestinationFull();
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
deleted file mode 100644
index d2e2946..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-public interface CommunicationsInput {
-
- InputStream getInputStream() throws IOException;
-
- long getBytesRead();
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
deleted file mode 100644
index 95cab29..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-public interface CommunicationsOutput {
-
- OutputStream getOutputStream() throws IOException;
-
- long getBytesWritten();
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
deleted file mode 100644
index d009cec..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public interface CommunicationsSession extends Closeable {
-
- public static final byte[] MAGIC_BYTES = {(byte) 'N', (byte) 'i', (byte) 'F', (byte) 'i'};
-
- CommunicationsInput getInput();
-
- CommunicationsOutput getOutput();
-
- void setTimeout(int millis) throws IOException;
-
- int getTimeout() throws IOException;
-
- void setUri(String uri);
-
- String getUri();
-
- String getUserDn();
-
- void setUserDn(String dn);
-
- boolean isDataAvailable();
-
- long getBytesWritten();
-
- long getBytesRead();
-
- /**
- * Asynchronously interrupts this FlowFileCodec. Implementations must ensure
- * that they stop sending and receiving data as soon as possible after this
- * method has been called, even if doing so results in sending only partial
- * data to the peer. This will usually result in the peer throwing a
- * SocketTimeoutException.
- */
- void interrupt();
-
- /**
- * Returns <code>true</code> if the connection is closed, <code>false</code>
- * otherwise.
- *
- * @return
- */
- boolean isClosed();
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
deleted file mode 100644
index 41334fe..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote.protocol;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-public enum RequestType {
-
- NEGOTIATE_FLOWFILE_CODEC,
- REQUEST_PEER_LIST,
- SEND_FLOWFILES,
- RECEIVE_FLOWFILES,
- SHUTDOWN;
-
- public void writeRequestType(final DataOutputStream dos) throws IOException {
- dos.writeUTF(name());
- }
-
- public static RequestType readRequestType(final DataInputStream dis) throws IOException {
- final String requestTypeVal = dis.readUTF();
- try {
- return RequestType.valueOf(requestTypeVal);
- } catch (final Exception e) {
- throw new IOException("Could not determine RequestType: received invalid value " + requestTypeVal);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
index 0d18f2e..0118534 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
@@ -18,13 +18,13 @@ package org.apache.nifi.remote.protocol;
import java.io.IOException;
-import org.apache.nifi.cluster.NodeInformant;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.remote.Peer;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.remote.VersionedRemoteResource;
+import org.apache.nifi.remote.cluster.NodeInformant;
import org.apache.nifi.remote.codec.FlowFileCodec;
import org.apache.nifi.remote.exception.BadRequestException;
import org.apache.nifi.remote.exception.HandshakeException;
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 2c1b085..bfa3d25 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -18,21 +18,10 @@ package org.apache.nifi.remote;
import static java.util.Objects.requireNonNull;
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
-import java.nio.channels.SocketChannel;
-import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Date;
import java.util.HashMap;
@@ -49,11 +38,8 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.regex.Pattern;
import javax.net.ssl.SSLContext;
-import javax.security.cert.CertificateExpiredException;
-import javax.security.cert.CertificateNotYetValidException;
import javax.ws.rs.core.Response;
import org.apache.nifi.connectable.ConnectableType;
@@ -72,16 +58,7 @@ import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.ProcessGroupCounts;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.remote.exception.BadRequestException;
-import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.PortNotRunningException;
-import org.apache.nifi.remote.exception.UnknownPortException;
-import org.apache.nifi.remote.io.socket.SocketChannelCommunicationsSession;
-import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
-import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelCommunicationsSession;
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-import org.apache.nifi.remote.protocol.socket.SocketClientProtocol;
+import org.apache.nifi.remote.client.socket.EndpointConnectionStatePool;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.util.FormatUtils;
@@ -156,9 +133,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
private volatile String authorizationIssue;
- private volatile PeerStatusCache peerStatusCache;
- private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
-
+ private final EndpointConnectionStatePool endpointConnectionPool;
private final ScheduledExecutorService backgroundThreadExecutor;
public StandardRemoteProcessGroup(final String id, final String targetUri, final ProcessGroup processGroup,
@@ -218,45 +193,23 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
}
};
- try {
- final File peersFile = getPeerPersistenceFile();
- this.peerStatusCache = new PeerStatusCache(recoverPersistedPeerStatuses(peersFile), peersFile.lastModified());
- } catch (final IOException e) {
- logger.error("{} Failed to recover persisted Peer Statuses due to {}", this, e);
- }
+ endpointConnectionPool = new EndpointConnectionStatePool(sslContext, eventReporter, getPeerPersistenceFile());
final Runnable refreshPeers = new Runnable() {
@Override
public void run() {
- final PeerStatusCache existingCache = peerStatusCache;
- if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
- return;
- }
-
- Set<RemoteGroupPort> ports = getInputPorts();
- if (ports.isEmpty()) {
- ports = getOutputPorts();
- }
-
- if (ports.isEmpty()){
- return;
- }
-
- // it doesn't really matter which port we use. Since we are just getting the Peer Status,
- // if the server indicates that the port cannot receive data for whatever reason, we will
- // simply ignore the error.
- final RemoteGroupPort port = ports.iterator().next();
-
- try {
- final Set<PeerStatus> statuses = fetchRemotePeerStatuses(port);
- peerStatusCache = new PeerStatusCache(statuses);
- logger.info("{} Successfully refreshed Peer Status; remote instance consists of {} peers", StandardRemoteProcessGroup.this, statuses.size());
- } catch (Exception e) {
- logger.warn("{} Unable to refresh Remote Group's peers due to {}", StandardRemoteProcessGroup.this, e);
- if (logger.isDebugEnabled()) {
- logger.warn("", e);
- }
- }
+ final boolean secure;
+ try {
+ secure = isSecure();
+ } catch (CommunicationsException e) {
+ logger.warn("{} Unable to determine if remote instance {} is configured for secure site-to-site due to {}; will not refresh list of peers", new Object[] {this, getTargetUri(), e.toString()});
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", e);
+ }
+ return;
+ }
+
+ endpointConnectionPool.refreshPeers(getTargetUri(), secure);
}
};
@@ -1255,52 +1208,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
}
}
- @Override
- public CommunicationsSession establishSiteToSiteConnection() throws IOException {
- final URI uri = apiUri;
- final String destinationUri = uri.toString();
- CommunicationsSession commsSession = null;
- try {
- if (isSecure()) {
- if (sslContext == null) {
- throw new IOException("Unable to communicate with " + getTargetUri() + " because it requires Secure Site-to-Site communications, but this instance is not configured for secure communications");
- }
-
- final Integer listeningPort = getListeningPort();
- if (listeningPort == null) {
- throw new IOException("Remote instance is not configured to allow incoming Site-to-Site connections");
- }
-
- final SSLSocketChannel socketChannel = new SSLSocketChannel(sslContext, uri.getHost(), listeningPort, true);
- socketChannel.connect();
- commsSession = new SSLSocketChannelCommunicationsSession(socketChannel, destinationUri);
-
- try {
- commsSession.setUserDn(socketChannel.getDn());
- } catch (final CertificateNotYetValidException | CertificateExpiredException ex) {
- throw new IOException(ex);
- }
- } else {
- final SocketChannel socketChannel = SocketChannel.open(new InetSocketAddress(uri.getHost(), getListeningPort()));
-
- commsSession = new SocketChannelCommunicationsSession(socketChannel, destinationUri);
- }
-
- commsSession.getOutput().getOutputStream().write(CommunicationsSession.MAGIC_BYTES);
-
- commsSession.setUri("nifi://" + uri.getHost() + ":" + uri.getPort());
- } catch (final IOException e) {
- if (commsSession != null) {
- try {
- commsSession.close();
- } catch (final IOException ignore) {
- }
- }
-
- throw e;
- }
- return commsSession;
- }
@Override
public EventReporter getEventReporter() {
@@ -1489,133 +1396,13 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
@Override
public Set<PeerStatus> getPeerStatuses() {
- final PeerStatusCache cache = this.peerStatusCache;
- if (cache == null || cache.getStatuses() == null || cache.getStatuses().isEmpty()) {
- return null;
- }
-
- if (cache.getTimestamp() + PEER_CACHE_MILLIS < System.currentTimeMillis()) {
- final Set<PeerStatus> equalizedSet = new HashSet<>(cache.getStatuses().size());
- for (final PeerStatus status : cache.getStatuses()) {
- final PeerStatus equalizedStatus = new PeerStatus(status.getHostname(), status.getPort(), status.isSecure(), 1);
- equalizedSet.add(equalizedStatus);
- }
-
- return equalizedSet;
- }
-
- return cache.getStatuses();
+ return endpointConnectionPool.getPeerStatuses();
}
- private Set<PeerStatus> fetchRemotePeerStatuses(final RemoteGroupPort port) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException, BadRequestException {
- final CommunicationsSession commsSession = establishSiteToSiteConnection();
- final Peer peer = new Peer(commsSession, "nifi://" + getTargetUri().getHost() + ":" + getListeningPort());
- final SocketClientProtocol clientProtocol = new SocketClientProtocol();
- clientProtocol.setPort(port);
- final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
- final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
- try {
- RemoteResourceFactory.initiateResourceNegotiation(clientProtocol, dis, dos);
- } catch (final HandshakeException e) {
- throw new BadRequestException(e.toString());
- }
-
- clientProtocol.handshake(peer);
- final Set<PeerStatus> peerStatuses = clientProtocol.getPeerStatuses(peer);
- persistPeerStatuses(peerStatuses);
-
- try {
- clientProtocol.shutdown(peer);
- } catch (final IOException e) {
- final String message = String.format("%s Failed to shutdown protocol when updating list of peers due to %s", this, e.toString());
- logger.warn(message);
- if (logger.isDebugEnabled()) {
- logger.warn("", e);
- }
- getEventReporter().reportEvent(Severity.WARNING, "Site to Site", message);
- }
-
- try {
- peer.close();
- } catch (final IOException e) {
- final String message = String.format("%s Failed to close resources when updating list of peers due to %s", this, e.toString());
- logger.warn(message);
- if (logger.isDebugEnabled()) {
- logger.warn("", e);
- }
- getEventReporter().reportEvent(Severity.WARNING, "Site to Site", message);
- }
-
- return peerStatuses;
- }
private File getPeerPersistenceFile() {
final File stateDir = NiFiProperties.getInstance().getPersistentStateDirectory();
return new File(stateDir, getIdentifier() + ".peers");
}
- private void persistPeerStatuses(final Set<PeerStatus> statuses) {
- final File peersFile = getPeerPersistenceFile();
- try (final OutputStream fos = new FileOutputStream(peersFile);
- final OutputStream out = new BufferedOutputStream(fos)) {
-
- for (final PeerStatus status : statuses) {
- final String line = status.getHostname() + ":" + status.getPort() + ":" + status.isSecure() + "\n";
- out.write(line.getBytes(StandardCharsets.UTF_8));
- }
-
- } catch (final IOException e) {
- logger.error("Failed to persist list of Peers due to {}; if restarted and peer's NCM is down, may be unable to transfer data until communications with NCM are restored", e.toString(), e);
- }
- }
-
- private Set<PeerStatus> recoverPersistedPeerStatuses(final File file) throws IOException {
- if (!file.exists()) {
- return null;
- }
-
- final Set<PeerStatus> statuses = new HashSet<>();
- try (final InputStream fis = new FileInputStream(file);
- final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
-
- String line;
- while ((line = reader.readLine()) != null) {
- final String[] splits = line.split(Pattern.quote(":"));
- if (splits.length != 3) {
- continue;
- }
-
- final String hostname = splits[0];
- final int port = Integer.parseInt(splits[1]);
- final boolean secure = Boolean.parseBoolean(splits[2]);
-
- statuses.add(new PeerStatus(hostname, port, secure, 1));
- }
- }
-
- return statuses;
- }
-
- private static class PeerStatusCache {
-
- private final Set<PeerStatus> statuses;
- private final long timestamp;
-
- public PeerStatusCache(final Set<PeerStatus> statuses) {
- this(statuses, System.currentTimeMillis());
- }
-
- public PeerStatusCache(final Set<PeerStatus> statuses, final long timestamp) {
- this.statuses = statuses;
- this.timestamp = timestamp;
- }
-
- public Set<PeerStatus> getStatuses() {
- return statuses;
- }
-
- public long getTimestamp() {
- return timestamp;
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/pom.xml b/nifi/nar-bundles/framework-bundle/framework/site-to-site/pom.xml
index 1c8d2c0..a7909a3 100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/pom.xml
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/pom.xml
@@ -62,6 +62,10 @@
<artifactId>nifi-utils</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>site-to-site-client</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</dependency>
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
deleted file mode 100644
index 4babb92..0000000
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.remote;
-
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-
-public abstract class AbstractCommunicationsSession implements CommunicationsSession {
- private String userDn;
-
- private volatile String uri;
-
- public AbstractCommunicationsSession(final String uri) {
- this.uri = uri;
- }
-
- @Override
- public String toString() {
- return uri;
- }
-
- @Override
- public void setUri(final String uri) {
- this.uri = uri;
- }
-
- @Override
- public String getUri() {
- return uri;
- }
-
- @Override
- public String getUserDn() {
- return userDn;
- }
-
- @Override
- public void setUserDn(final String dn) {
- this.userDn = dn;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
index 49d3c3c..922d4e7 100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
@@ -26,48 +26,8 @@ import org.apache.nifi.remote.exception.HandshakeException;
import org.apache.nifi.remote.protocol.ClientProtocol;
import org.apache.nifi.remote.protocol.ServerProtocol;
-public class RemoteResourceFactory {
+public class RemoteResourceFactory extends RemoteResourceInitiator {
- public static final int RESOURCE_OK = 20;
- public static final int DIFFERENT_RESOURCE_VERSION = 21;
- public static final int ABORT = 255;
-
-
- public static VersionedRemoteResource initiateResourceNegotiation(final VersionedRemoteResource resource, final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException {
- // Write the classname of the RemoteStreamCodec, followed by its version
- dos.writeUTF(resource.getResourceName());
- final VersionNegotiator negotiator = resource.getVersionNegotiator();
- dos.writeInt(negotiator.getVersion());
- dos.flush();
-
- // wait for response from server.
- final int statusCode = dis.read();
- switch (statusCode) {
- case RESOURCE_OK: // server accepted our proposal of codec name/version
- return resource;
- case DIFFERENT_RESOURCE_VERSION: // server accepted our proposal of codec name but not the version
- // Get server's preferred version
- final int newVersion = dis.readInt();
-
- // Determine our new preferred version that is no greater than the server's preferred version.
- final Integer newPreference = negotiator.getPreferredVersion(newVersion);
- // If we could not agree with server on a version, fail now.
- if ( newPreference == null ) {
- throw new HandshakeException("Could not agree on version for " + resource);
- }
-
- negotiator.setVersion(newPreference);
-
- // Attempt negotiation of resource based on our new preferred version.
- return initiateResourceNegotiation(resource, dis, dos);
- case ABORT:
- throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF());
- default:
- return null; // Unable to negotiate codec
- }
- }
-
-
@SuppressWarnings("unchecked")
public static <T extends FlowFileCodec> T receiveCodecNegotiation(final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException {
final String codecName = dis.readUTF();
[4/5] incubator-nifi git commit: NIFI-282: Begin refactoring and
creating client
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
new file mode 100644
index 0000000..d18a4ee
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
@@ -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.remote.codec;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.remote.StandardVersionNegotiator;
+import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.remote.exception.ProtocolException;
+
+public class StandardFlowFileCodec implements FlowFileCodec {
+ public static final int MAX_NUM_ATTRIBUTES = 25000;
+
+ public static final String DEFAULT_FLOWFILE_PATH = "./";
+
+ private final VersionNegotiator versionNegotiator;
+
+ public StandardFlowFileCodec() {
+ versionNegotiator = new StandardVersionNegotiator(1);
+ }
+
+ @Override
+ public FlowFile encode(final FlowFile flowFile, final ProcessSession session, final OutputStream encodedOut) throws IOException {
+ final DataOutputStream out = new DataOutputStream(encodedOut);
+
+ final Map<String, String> attributes = flowFile.getAttributes();
+ out.writeInt(attributes.size());
+ for ( final Map.Entry<String, String> entry : attributes.entrySet() ) {
+ writeString(entry.getKey(), out);
+ writeString(entry.getValue(), out);
+ }
+
+ out.writeLong(flowFile.getSize());
+
+ session.read(flowFile, new InputStreamCallback() {
+ @Override
+ public void process(final InputStream in) throws IOException {
+ final byte[] buffer = new byte[8192];
+ int len;
+ while ( (len = in.read(buffer)) > 0 ) {
+ encodedOut.write(buffer, 0, len);
+ }
+
+ encodedOut.flush();
+ }
+ });
+
+ return flowFile;
+ }
+
+
+ @Override
+ public FlowFile decode(final InputStream stream, final ProcessSession session) throws IOException, ProtocolException {
+ final DataInputStream in = new DataInputStream(stream);
+
+ final int numAttributes;
+ try {
+ numAttributes = in.readInt();
+ } catch (final EOFException e) {
+ // we're out of data.
+ return null;
+ }
+
+ // This is here because if the stream is not properly formed, we could get up to Integer.MAX_VALUE attributes, which will
+ // generally result in an OutOfMemoryError.
+ if ( numAttributes > MAX_NUM_ATTRIBUTES ) {
+ throw new ProtocolException("FlowFile exceeds maximum number of attributes with a total of " + numAttributes);
+ }
+
+ try {
+ final Map<String, String> attributes = new HashMap<>(numAttributes);
+ for (int i=0; i < numAttributes; i++) {
+ final String attrName = readString(in);
+ final String attrValue = readString(in);
+ attributes.put(attrName, attrValue);
+ }
+
+ final long numBytes = in.readLong();
+
+ FlowFile flowFile = session.create();
+ flowFile = session.putAllAttributes(flowFile, attributes);
+ flowFile = session.write(flowFile, new OutputStreamCallback() {
+ @Override
+ public void process(final OutputStream out) throws IOException {
+ int len;
+ long size = 0;
+ final byte[] buffer = new byte[8192];
+
+ while ( size < numBytes && (len = in.read(buffer, 0, (int) Math.min(buffer.length, numBytes - size))) > 0 ) {
+ out.write(buffer, 0, len);
+ size += len;
+ }
+
+ if ( size != numBytes ) {
+ throw new EOFException("Expected " + numBytes + " bytes but received only " + size);
+ }
+ }
+ });
+
+ return flowFile;
+ } catch (final EOFException e) {
+ session.rollback();
+
+ // we throw the general IOException here because we did not expect to hit EOFException
+ throw e;
+ }
+ }
+
+ private void writeString(final String val, final DataOutputStream out) throws IOException {
+ final byte[] bytes = val.getBytes("UTF-8");
+ out.writeInt(bytes.length);
+ out.write(bytes);
+ }
+
+
+ private String readString(final DataInputStream in) throws IOException {
+ final int numBytes = in.readInt();
+ final byte[] bytes = new byte[numBytes];
+ StreamUtils.fillBuffer(in, bytes, true);
+ return new String(bytes, "UTF-8");
+ }
+
+ @Override
+ public List<Integer> getSupportedVersions() {
+ return versionNegotiator.getSupportedVersions();
+ }
+
+ @Override
+ public VersionNegotiator getVersionNegotiator() {
+ return versionNegotiator;
+ }
+
+ @Override
+ public String toString() {
+ return "Standard FlowFile Codec, Version " + versionNegotiator.getVersion();
+ }
+
+ @Override
+ public String getResourceName() {
+ return "StandardFlowFileCodec";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
new file mode 100644
index 0000000..f6c2f4f
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/BadRequestException.java
@@ -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.remote.exception;
+
+public class BadRequestException extends Exception {
+
+ private static final long serialVersionUID = -8034602852256106560L;
+
+ public BadRequestException(final String message) {
+ super(message);
+ }
+
+ public BadRequestException(final Throwable cause) {
+ super(cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
new file mode 100644
index 0000000..b61fc65
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
@@ -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.remote.exception;
+
+public class HandshakeException extends Exception {
+
+ private static final long serialVersionUID = 178192341908726L;
+
+ public HandshakeException(final String message) {
+ super(message);
+ }
+
+ public HandshakeException(final Throwable cause) {
+ super(cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
new file mode 100644
index 0000000..24ff3a5
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/NotAuthorizedException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.exception;
+
+public class NotAuthorizedException extends Exception {
+
+ private static final long serialVersionUID = 2952623568114035498L;
+
+ public NotAuthorizedException(final String message) {
+ super(message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
new file mode 100644
index 0000000..af0f467
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.exception;
+
+public class PortNotRunningException extends Exception {
+
+ private static final long serialVersionUID = -2790940982005516375L;
+
+ public PortNotRunningException(final String message) {
+ super(message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
new file mode 100644
index 0000000..0f50b98
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.exception;
+
+public class ProtocolException extends Exception {
+
+ private static final long serialVersionUID = 5763900324505818495L;
+
+ public ProtocolException(final String message, final Throwable cause) {
+ super(message, cause);
+ }
+
+ public ProtocolException(final String message) {
+ super(message);
+ }
+
+ public ProtocolException(final Throwable cause) {
+ super(cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
new file mode 100644
index 0000000..dd675b3
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/RequestExpiredException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.exception;
+
+/**
+ * Used to indicate that by the time the request was serviced, it had already
+ * expired
+ */
+public class RequestExpiredException extends Exception {
+
+ private static final long serialVersionUID = -7037025330562827852L;
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
new file mode 100644
index 0000000..e6a0fe7
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.exception;
+
+public class UnknownPortException extends Exception {
+
+ private static final long serialVersionUID = -2790940982005516375L;
+
+ public UnknownPortException(final String message) {
+ super(message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java
new file mode 100644
index 0000000..926809c
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.exception;
+
+import org.apache.nifi.remote.codec.FlowFileCodec;
+
+public class UnsupportedCodecException extends RuntimeException {
+ private static final long serialVersionUID = 198234789237L;
+
+ public UnsupportedCodecException(final String codecName) {
+ super("Codec " + codecName + " is not supported");
+ }
+
+ public UnsupportedCodecException(final FlowFileCodec codec, final int version) {
+ super("Codec " + codec.getClass().getName() + " does not support Version " + version);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
new file mode 100644
index 0000000..0822b6a
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.io.socket;
+
+import java.io.IOException;
+import java.nio.channels.SocketChannel;
+
+import org.apache.nifi.remote.AbstractCommunicationsSession;
+
+public class SocketChannelCommunicationsSession extends AbstractCommunicationsSession {
+ private final SocketChannel channel;
+ private final SocketChannelInput request;
+ private final SocketChannelOutput response;
+ private int timeout = 30000;
+
+ public SocketChannelCommunicationsSession(final SocketChannel socketChannel, final String uri) throws IOException {
+ super(uri);
+ request = new SocketChannelInput(socketChannel);
+ response = new SocketChannelOutput(socketChannel);
+ channel = socketChannel;
+ socketChannel.configureBlocking(false);
+ }
+
+ @Override
+ public boolean isClosed() {
+ return !channel.isConnected();
+ }
+
+ @Override
+ public SocketChannelInput getInput() {
+ return request;
+ }
+
+ @Override
+ public SocketChannelOutput getOutput() {
+ return response;
+ }
+
+ @Override
+ public void setTimeout(final int millis) throws IOException {
+ request.setTimeout(millis);
+ response.setTimeout(millis);
+ this.timeout = millis;
+ }
+
+ @Override
+ public int getTimeout() throws IOException {
+ return timeout;
+ }
+
+ @Override
+ public void close() throws IOException {
+ channel.close();
+ }
+
+ @Override
+ public boolean isDataAvailable() {
+ return request.isDataAvailable();
+ }
+
+ @Override
+ public long getBytesWritten() {
+ return response.getBytesWritten();
+ }
+
+ @Override
+ public long getBytesRead() {
+ return request.getBytesRead();
+ }
+
+ @Override
+ public void interrupt() {
+ request.interrupt();
+ response.interrupt();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
new file mode 100644
index 0000000..9e451fd
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
@@ -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.remote.io.socket;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.channels.SocketChannel;
+
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.stream.io.ByteCountingInputStream;
+import org.apache.nifi.remote.io.InterruptableInputStream;
+import org.apache.nifi.remote.protocol.CommunicationsInput;
+
+public class SocketChannelInput implements CommunicationsInput {
+ private final SocketChannelInputStream socketIn;
+ private final ByteCountingInputStream countingIn;
+ private final InputStream bufferedIn;
+ private final InterruptableInputStream interruptableIn;
+
+ public SocketChannelInput(final SocketChannel socketChannel) throws IOException {
+ this.socketIn = new SocketChannelInputStream(socketChannel);
+ countingIn = new ByteCountingInputStream(socketIn);
+ bufferedIn = new BufferedInputStream(countingIn);
+ interruptableIn = new InterruptableInputStream(bufferedIn);
+ }
+
+ @Override
+ public InputStream getInputStream() throws IOException {
+ return interruptableIn;
+ }
+
+ public void setTimeout(final int millis) {
+ socketIn.setTimeout(millis);
+ }
+
+ public boolean isDataAvailable() {
+ try {
+ return interruptableIn.available() > 0;
+ } catch (final Exception e) {
+ return false;
+ }
+ }
+
+ @Override
+ public long getBytesRead() {
+ return countingIn.getBytesRead();
+ }
+
+ public void interrupt() {
+ interruptableIn.interrupt();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
new file mode 100644
index 0000000..26c0164
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.io.socket;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.channels.SocketChannel;
+
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.ByteCountingOutputStream;
+import org.apache.nifi.remote.io.InterruptableOutputStream;
+import org.apache.nifi.remote.protocol.CommunicationsOutput;
+
+public class SocketChannelOutput implements CommunicationsOutput {
+ private final SocketChannelOutputStream socketOutStream;
+ private final ByteCountingOutputStream countingOut;
+ private final OutputStream bufferedOut;
+ private final InterruptableOutputStream interruptableOut;
+
+ public SocketChannelOutput(final SocketChannel socketChannel) throws IOException {
+ socketOutStream = new SocketChannelOutputStream(socketChannel);
+ countingOut = new ByteCountingOutputStream(socketOutStream);
+ bufferedOut = new BufferedOutputStream(countingOut);
+ interruptableOut = new InterruptableOutputStream(bufferedOut);
+ }
+
+ @Override
+ public OutputStream getOutputStream() throws IOException {
+ return interruptableOut;
+ }
+
+ public void setTimeout(final int timeout) {
+ socketOutStream.setTimeout(timeout);
+ }
+
+ @Override
+ public long getBytesWritten() {
+ return countingOut.getBytesWritten();
+ }
+
+ public void interrupt() {
+ interruptableOut.interrupt();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
new file mode 100644
index 0000000..dca1d84
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.io.socket.ssl;
+
+import java.io.IOException;
+
+import org.apache.nifi.remote.AbstractCommunicationsSession;
+
+public class SSLSocketChannelCommunicationsSession extends AbstractCommunicationsSession {
+ private final SSLSocketChannel channel;
+ private final SSLSocketChannelInput request;
+ private final SSLSocketChannelOutput response;
+
+ public SSLSocketChannelCommunicationsSession(final SSLSocketChannel channel, final String uri) {
+ super(uri);
+ request = new SSLSocketChannelInput(channel);
+ response = new SSLSocketChannelOutput(channel);
+ this.channel = channel;
+ }
+
+ @Override
+ public SSLSocketChannelInput getInput() {
+ return request;
+ }
+
+ @Override
+ public SSLSocketChannelOutput getOutput() {
+ return response;
+ }
+
+ @Override
+ public void setTimeout(final int millis) throws IOException {
+ channel.setTimeout(millis);
+ }
+
+ @Override
+ public int getTimeout() throws IOException {
+ return channel.getTimeout();
+ }
+
+ @Override
+ public void close() throws IOException {
+ channel.close();
+ }
+
+ @Override
+ public boolean isClosed() {
+ return channel.isClosed();
+ }
+
+ @Override
+ public boolean isDataAvailable() {
+ try {
+ return request.isDataAvailable();
+ } catch (final Exception e) {
+ return false;
+ }
+ }
+
+ @Override
+ public long getBytesWritten() {
+ return response.getBytesWritten();
+ }
+
+ @Override
+ public long getBytesRead() {
+ return request.getBytesRead();
+ }
+
+ @Override
+ public void interrupt() {
+ channel.interrupt();
+ }
+
+ @Override
+ public String toString() {
+ return super.toString() + "[SSLSocketChannel=" + channel + "]";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
new file mode 100644
index 0000000..60ef33f
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
@@ -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.remote.io.socket.ssl;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.stream.io.ByteCountingInputStream;
+import org.apache.nifi.remote.protocol.CommunicationsInput;
+
+public class SSLSocketChannelInput implements CommunicationsInput {
+ private final SSLSocketChannelInputStream in;
+ private final ByteCountingInputStream countingIn;
+ private final InputStream bufferedIn;
+
+ public SSLSocketChannelInput(final SSLSocketChannel socketChannel) {
+ in = new SSLSocketChannelInputStream(socketChannel);
+ countingIn = new ByteCountingInputStream(in);
+ this.bufferedIn = new BufferedInputStream(countingIn);
+ }
+
+ @Override
+ public InputStream getInputStream() throws IOException {
+ return bufferedIn;
+ }
+
+ public boolean isDataAvailable() throws IOException {
+ return bufferedIn.available() > 0;
+ }
+
+ @Override
+ public long getBytesRead() {
+ return countingIn.getBytesRead();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
new file mode 100644
index 0000000..dc3d68f
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
@@ -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.remote.io.socket.ssl;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.ByteCountingOutputStream;
+import org.apache.nifi.remote.protocol.CommunicationsOutput;
+
+public class SSLSocketChannelOutput implements CommunicationsOutput {
+ private final OutputStream out;
+ private final ByteCountingOutputStream countingOut;
+
+ public SSLSocketChannelOutput(final SSLSocketChannel channel) {
+ countingOut = new ByteCountingOutputStream(new SSLSocketChannelOutputStream(channel));
+ out = new BufferedOutputStream(countingOut);
+ }
+
+ @Override
+ public OutputStream getOutputStream() throws IOException {
+ return out;
+ }
+
+ @Override
+ public long getBytesWritten() {
+ return countingOut.getBytesWritten();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
new file mode 100644
index 0000000..32274eb
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.protocol;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.remote.Peer;
+import org.apache.nifi.remote.PeerStatus;
+import org.apache.nifi.remote.VersionedRemoteResource;
+import org.apache.nifi.remote.codec.FlowFileCodec;
+import org.apache.nifi.remote.exception.HandshakeException;
+import org.apache.nifi.remote.exception.PortNotRunningException;
+import org.apache.nifi.remote.exception.ProtocolException;
+import org.apache.nifi.remote.exception.UnknownPortException;
+
+public interface ClientProtocol extends VersionedRemoteResource {
+
+ void handshake(Peer peer) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException;
+
+ Set<PeerStatus> getPeerStatuses(Peer currentPeer) throws IOException, ProtocolException;
+
+ FlowFileCodec negotiateCodec(Peer peer) throws IOException, ProtocolException;
+
+ void receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
+
+ void transferFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
+
+ void shutdown(Peer peer) throws IOException, ProtocolException;
+
+ boolean isReadyForFileTransfer();
+
+ /**
+ * returns <code>true</code> if remote instance indicates that the port is
+ * invalid
+ *
+ * @return
+ * @throws IllegalStateException if a handshake has not successfully
+ * completed
+ */
+ boolean isPortInvalid() throws IllegalStateException;
+
+ /**
+ * returns <code>true</code> if remote instance indicates that the port is
+ * unknown
+ *
+ * @return
+ * @throws IllegalStateException if a handshake has not successfully
+ * completed
+ */
+ boolean isPortUnknown();
+
+ /**
+ * returns <code>true</code> if remote instance indicates that the port's
+ * destination is full
+ *
+ * @return
+ * @throws IllegalStateException if a handshake has not successfully
+ * completed
+ */
+ boolean isDestinationFull();
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
new file mode 100644
index 0000000..d2e2946
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
@@ -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.remote.protocol;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public interface CommunicationsInput {
+
+ InputStream getInputStream() throws IOException;
+
+ long getBytesRead();
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
new file mode 100644
index 0000000..95cab29
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
@@ -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.remote.protocol;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+public interface CommunicationsOutput {
+
+ OutputStream getOutputStream() throws IOException;
+
+ long getBytesWritten();
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
new file mode 100644
index 0000000..d009cec
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.protocol;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public interface CommunicationsSession extends Closeable {
+
+ public static final byte[] MAGIC_BYTES = {(byte) 'N', (byte) 'i', (byte) 'F', (byte) 'i'};
+
+ CommunicationsInput getInput();
+
+ CommunicationsOutput getOutput();
+
+ void setTimeout(int millis) throws IOException;
+
+ int getTimeout() throws IOException;
+
+ void setUri(String uri);
+
+ String getUri();
+
+ String getUserDn();
+
+ void setUserDn(String dn);
+
+ boolean isDataAvailable();
+
+ long getBytesWritten();
+
+ long getBytesRead();
+
+ /**
+ * Asynchronously interrupts this FlowFileCodec. Implementations must ensure
+ * that they stop sending and receiving data as soon as possible after this
+ * method has been called, even if doing so results in sending only partial
+ * data to the peer. This will usually result in the peer throwing a
+ * SocketTimeoutException.
+ */
+ void interrupt();
+
+ /**
+ * Returns <code>true</code> if the connection is closed, <code>false</code>
+ * otherwise.
+ *
+ * @return
+ */
+ boolean isClosed();
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
new file mode 100644
index 0000000..41334fe
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.protocol;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public enum RequestType {
+
+ NEGOTIATE_FLOWFILE_CODEC,
+ REQUEST_PEER_LIST,
+ SEND_FLOWFILES,
+ RECEIVE_FLOWFILES,
+ SHUTDOWN;
+
+ public void writeRequestType(final DataOutputStream dos) throws IOException {
+ dos.writeUTF(name());
+ }
+
+ public static RequestType readRequestType(final DataInputStream dis) throws IOException {
+ final String requestTypeVal = dis.readUTF();
+ try {
+ return RequestType.valueOf(requestTypeVal);
+ } catch (final Exception e) {
+ throw new IOException("Could not determine RequestType: received invalid value " + requestTypeVal);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
new file mode 100644
index 0000000..c4519cd
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.protocol.socket;
+
+public enum HandshakeProperty {
+ GZIP,
+ PORT_IDENTIFIER,
+ REQUEST_EXPIRATION_MILLIS;
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
new file mode 100644
index 0000000..eae1940
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
@@ -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.remote.protocol.socket;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.nifi.remote.exception.ProtocolException;
+
+public class Response {
+ private final ResponseCode code;
+ private final String message;
+
+ private Response(final ResponseCode code, final String explanation) {
+ this.code = code;
+ this.message = explanation;
+ }
+
+ public ResponseCode getCode() {
+ return code;
+ }
+
+ public String getMessage() {
+ return message;
+ }
+
+ public static Response read(final DataInputStream in) throws IOException, ProtocolException {
+ final ResponseCode code = ResponseCode.readCode(in);
+ final String message = code.containsMessage() ? in.readUTF() : null;
+ return new Response(code, message);
+ }
+
+ @Override
+ public String toString() {
+ return code + ": " + message;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
new file mode 100644
index 0000000..0e588cd
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
@@ -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.remote.protocol.socket;
+
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.nifi.remote.exception.ProtocolException;
+
+
+public enum ResponseCode {
+ RESERVED(0, "Reserved for Future Use", false), // This will likely be used if we ever need to expand the length of
+ // ResponseCode, so that we can indicate a 0 followed by some other bytes
+
+ // handshaking properties
+ PROPERTIES_OK(1, "Properties OK", false),
+ UNKNOWN_PROPERTY_NAME(230, "Unknown Property Name", true),
+ ILLEGAL_PROPERTY_VALUE(231, "Illegal Property Value", true),
+ MISSING_PROPERTY(232, "Missing Property", true),
+
+ // transaction indicators
+ CONTINUE_TRANSACTION(10, "Continue Transaction", false),
+ FINISH_TRANSACTION(11, "Finish Transaction", false),
+ CONFIRM_TRANSACTION(12, "Confirm Transaction", true), // "Explanation" of this code is the checksum
+ TRANSACTION_FINISHED(13, "Transaction Finished", false),
+ TRANSACTION_FINISHED_BUT_DESTINATION_FULL(14, "Transaction Finished But Destination is Full", false),
+ BAD_CHECKSUM(19, "Bad Checksum", false),
+
+ // data availability indicators
+ MORE_DATA(20, "More Data Exists", false),
+ NO_MORE_DATA(21, "No More Data Exists", false),
+
+ // port state indicators
+ UNKNOWN_PORT(200, "Unknown Port", false),
+ PORT_NOT_IN_VALID_STATE(201, "Port Not in a Valid State", true),
+ PORTS_DESTINATION_FULL(202, "Port's Destination is Full", false),
+
+ // authorization
+ UNAUTHORIZED(240, "User Not Authorized", true),
+
+ // error indicators
+ ABORT(250, "Abort", true),
+ UNRECOGNIZED_RESPONSE_CODE(254, "Unrecognized Response Code", false),
+ END_OF_STREAM(255, "End of Stream", false);
+
+ private static final ResponseCode[] codeArray = new ResponseCode[256];
+
+ static {
+ for ( final ResponseCode responseCode : ResponseCode.values() ) {
+ codeArray[responseCode.getCode()] = responseCode;
+ }
+ }
+
+ private static final byte CODE_SEQUENCE_VALUE_1 = (byte) 'R';
+ private static final byte CODE_SEQUENCE_VALUE_2 = (byte) 'C';
+ private final int code;
+ private final byte[] codeSequence;
+ private final String description;
+ private final boolean containsMessage;
+
+ private ResponseCode(final int code, final String description, final boolean containsMessage) {
+ this.codeSequence = new byte[] {CODE_SEQUENCE_VALUE_1, CODE_SEQUENCE_VALUE_2, (byte) code};
+ this.code = code;
+ this.description = description;
+ this.containsMessage = containsMessage;
+ }
+
+ public int getCode() {
+ return code;
+ }
+
+ public byte[] getCodeSequence() {
+ return codeSequence;
+ }
+
+ @Override
+ public String toString() {
+ return description;
+ }
+
+ public boolean containsMessage() {
+ return containsMessage;
+ }
+
+ public void writeResponse(final DataOutputStream out) throws IOException {
+ if ( containsMessage() ) {
+ throw new IllegalArgumentException("ResponseCode " + code + " expects an explanation");
+ }
+
+ out.write(getCodeSequence());
+ out.flush();
+ }
+
+ public void writeResponse(final DataOutputStream out, final String explanation) throws IOException {
+ if ( !containsMessage() ) {
+ throw new IllegalArgumentException("ResponseCode " + code + " does not expect an explanation");
+ }
+
+ out.write(getCodeSequence());
+ out.writeUTF(explanation);
+ out.flush();
+ }
+
+ static ResponseCode readCode(final InputStream in) throws IOException, ProtocolException {
+ final int byte1 = in.read();
+ if ( byte1 < 0 ) {
+ throw new EOFException();
+ } else if ( byte1 != CODE_SEQUENCE_VALUE_1 ) {
+ throw new ProtocolException("Expected to receive ResponseCode, but the stream did not have a ResponseCode");
+ }
+
+ final int byte2 = in.read();
+ if ( byte2 < 0 ) {
+ throw new EOFException();
+ } else if ( byte2 != CODE_SEQUENCE_VALUE_2 ) {
+ throw new ProtocolException("Expected to receive ResponseCode, but the stream did not have a ResponseCode");
+ }
+
+ final int byte3 = in.read();
+ if ( byte3 < 0 ) {
+ throw new EOFException();
+ }
+
+ final ResponseCode responseCode = codeArray[byte3];
+ if (responseCode == null) {
+ throw new ProtocolException("Received Response Code of " + byte3 + " but do not recognize this code");
+ }
+ return responseCode;
+ }
+
+ public static ResponseCode fromSequence(final byte[] value) {
+ final int code = value[3] & 0xFF;
+ final ResponseCode responseCode = codeArray[code];
+ return (responseCode == null) ? UNRECOGNIZED_RESPONSE_CODE : responseCode;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
new file mode 100644
index 0000000..2f4f755
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@ -0,0 +1,517 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.protocol.socket;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.zip.CRC32;
+import java.util.zip.CheckedInputStream;
+import java.util.zip.CheckedOutputStream;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.remote.Peer;
+import org.apache.nifi.remote.PeerStatus;
+import org.apache.nifi.remote.RemoteDestination;
+import org.apache.nifi.remote.RemoteResourceInitiator;
+import org.apache.nifi.remote.StandardVersionNegotiator;
+import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.remote.codec.FlowFileCodec;
+import org.apache.nifi.remote.codec.StandardFlowFileCodec;
+import org.apache.nifi.remote.exception.HandshakeException;
+import org.apache.nifi.remote.exception.ProtocolException;
+import org.apache.nifi.remote.io.CompressionInputStream;
+import org.apache.nifi.remote.io.CompressionOutputStream;
+import org.apache.nifi.remote.protocol.ClientProtocol;
+import org.apache.nifi.remote.protocol.CommunicationsSession;
+import org.apache.nifi.remote.protocol.RequestType;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SocketClientProtocol implements ClientProtocol {
+ private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1);
+
+
+ private RemoteDestination destination;
+ private boolean useCompression;
+
+ private String commsIdentifier;
+ private boolean handshakeComplete = false;
+
+ private final Logger logger = LoggerFactory.getLogger(SocketClientProtocol.class);
+
+ private Response handshakeResponse = null;
+ private boolean readyForFileTransfer = false;
+ private String transitUriPrefix = null;
+
+ private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
+
+ public SocketClientProtocol() {
+ }
+
+ public void setDestination(final RemoteDestination destination) {
+ this.destination = destination;
+ this.useCompression = destination.isUseCompression();
+ }
+
+
+ @Override
+ public void handshake(final Peer peer) throws IOException, HandshakeException {
+ handshake(peer, destination.getIdentifier(), (int) destination.getCommunicationsTimeout(TimeUnit.MILLISECONDS));
+ }
+
+ public void handshake(final Peer peer, final String destinationId, final int timeoutMillis) throws IOException, HandshakeException {
+ if ( handshakeComplete ) {
+ throw new IllegalStateException("Handshake has already been completed");
+ }
+ commsIdentifier = UUID.randomUUID().toString();
+ logger.debug("{} handshaking with {}", this, peer);
+
+ final Map<HandshakeProperty, String> properties = new HashMap<>();
+ properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression));
+
+ if ( destinationId != null ) {
+ properties.put(HandshakeProperty.PORT_IDENTIFIER, destination.getIdentifier());
+ }
+
+ properties.put(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, String.valueOf(timeoutMillis) );
+
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ commsSession.setTimeout((int) destination.getCommunicationsTimeout(TimeUnit.MILLISECONDS));
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ dos.writeUTF(commsIdentifier);
+
+ if ( versionNegotiator.getVersion() >= 3 ) {
+ dos.writeUTF(peer.getUrl());
+ transitUriPrefix = peer.getUrl();
+
+ if ( !transitUriPrefix.endsWith("/") ) {
+ transitUriPrefix = transitUriPrefix + "/";
+ }
+ }
+
+ dos.writeInt(properties.size());
+ for ( final Map.Entry<HandshakeProperty, String> entry : properties.entrySet() ) {
+ dos.writeUTF(entry.getKey().name());
+ dos.writeUTF(entry.getValue());
+ }
+
+ dos.flush();
+
+ try {
+ handshakeResponse = Response.read(dis);
+ } catch (final ProtocolException e) {
+ throw new HandshakeException(e);
+ }
+
+ switch (handshakeResponse.getCode()) {
+ case PORT_NOT_IN_VALID_STATE:
+ case UNKNOWN_PORT:
+ case PORTS_DESTINATION_FULL:
+ break;
+ case PROPERTIES_OK:
+ readyForFileTransfer = true;
+ break;
+ default:
+ logger.error("{} received unexpected response {} from {} when negotiating Codec", new Object[] {
+ this, handshakeResponse, peer});
+ peer.close();
+ throw new HandshakeException("Received unexpected response " + handshakeResponse);
+ }
+
+ logger.debug("{} Finished handshake with {}", this, peer);
+ handshakeComplete = true;
+ }
+
+ public boolean isReadyForFileTransfer() {
+ return readyForFileTransfer;
+ }
+
+ public boolean isPortInvalid() {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not completed successfully");
+ }
+ return handshakeResponse.getCode() == ResponseCode.PORT_NOT_IN_VALID_STATE;
+ }
+
+ public boolean isPortUnknown() {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not completed successfully");
+ }
+ return handshakeResponse.getCode() == ResponseCode.UNKNOWN_PORT;
+ }
+
+ public boolean isDestinationFull() {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not completed successfully");
+ }
+ return handshakeResponse.getCode() == ResponseCode.PORTS_DESTINATION_FULL;
+ }
+
+ @Override
+ public Set<PeerStatus> getPeerStatuses(final Peer peer) throws IOException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+
+ logger.debug("{} Get Peer Statuses from {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ RequestType.REQUEST_PEER_LIST.writeRequestType(dos);
+ dos.flush();
+ final int numPeers = dis.readInt();
+ final Set<PeerStatus> peers = new HashSet<>(numPeers);
+ for (int i=0; i < numPeers; i++) {
+ final String hostname = dis.readUTF();
+ final int port = dis.readInt();
+ final boolean secure = dis.readBoolean();
+ final int flowFileCount = dis.readInt();
+ peers.add(new PeerStatus(hostname, port, secure, flowFileCount));
+ }
+
+ logger.debug("{} Received {} Peer Statuses from {}", this, peers.size(), peer);
+ return peers;
+ }
+
+ @Override
+ public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+
+ logger.debug("{} Negotiating Codec with {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ RequestType.NEGOTIATE_FLOWFILE_CODEC.writeRequestType(dos);
+
+ FlowFileCodec codec = new StandardFlowFileCodec();
+ try {
+ codec = (FlowFileCodec) RemoteResourceInitiator.initiateResourceNegotiation(codec, dis, dos);
+ } catch (HandshakeException e) {
+ throw new ProtocolException(e.toString());
+ }
+ logger.debug("{} negotiated FlowFileCodec {} with {}", new Object[] {this, codec, commsSession});
+
+ return codec;
+ }
+
+
+ @Override
+ public void receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+ if ( !readyForFileTransfer ) {
+ throw new IllegalStateException("Cannot receive files; handshake resolution was " + handshakeResponse);
+ }
+
+ logger.debug("{} Receiving FlowFiles from {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ String userDn = commsSession.getUserDn();
+ if ( userDn == null ) {
+ userDn = "none";
+ }
+
+ // Indicate that we would like to have some data
+ RequestType.RECEIVE_FLOWFILES.writeRequestType(dos);
+ dos.flush();
+
+ // Determine if Peer will send us data or has no data to send us
+ final Response dataAvailableCode = Response.read(dis);
+ switch (dataAvailableCode.getCode()) {
+ case MORE_DATA:
+ logger.debug("{} {} Indicates that data is available", this, peer);
+ break;
+ case NO_MORE_DATA:
+ logger.debug("{} No data available from {}", peer);
+ return;
+ default:
+ throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
+ }
+
+ final StopWatch stopWatch = new StopWatch(true);
+ final Set<FlowFile> flowFilesReceived = new HashSet<>();
+ long bytesReceived = 0L;
+ final CRC32 crc = new CRC32();
+
+ // Peer has data. Decode the bytes into FlowFiles until peer says he's finished sending data.
+ boolean continueTransaction = true;
+ String calculatedCRC = "";
+ while (continueTransaction) {
+ final InputStream flowFileInputStream = useCompression ? new CompressionInputStream(dis) : dis;
+ final CheckedInputStream checkedIn = new CheckedInputStream(flowFileInputStream, crc);
+
+ final long startNanos = System.nanoTime();
+ FlowFile flowFile = codec.decode(checkedIn, session);
+ final long transmissionNanos = System.nanoTime() - startNanos;
+ final long transmissionMillis = TimeUnit.MILLISECONDS.convert(transmissionNanos, TimeUnit.NANOSECONDS);
+
+ final String sourceFlowFileIdentifier = flowFile.getAttribute(CoreAttributes.UUID.key());
+ flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+
+ final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceFlowFileIdentifier;
+ session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transmissionMillis);
+
+ session.transfer(flowFile, Relationship.ANONYMOUS);
+ bytesReceived += flowFile.getSize();
+ flowFilesReceived.add(flowFile);
+ logger.debug("{} Received {} from {}", this, flowFile, peer);
+
+ final Response transactionCode = Response.read(dis);
+ switch (transactionCode.getCode()) {
+ case CONTINUE_TRANSACTION:
+ logger.trace("{} Received ContinueTransaction indicator from {}", this, peer);
+ break;
+ case FINISH_TRANSACTION:
+ logger.trace("{} Received FinishTransaction indicator from {}", this, peer);
+ continueTransaction = false;
+ calculatedCRC = String.valueOf(checkedIn.getChecksum().getValue());
+ break;
+ default:
+ throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionCode);
+ }
+ }
+
+ // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
+ // to peer so that we can verify that the connection is still open. This is a two-phase commit,
+ // which helps to prevent the chances of data duplication. Without doing this, we may commit the
+ // session and then when we send the response back to the peer, the peer may have timed out and may not
+ // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
+ // Critical Section involved in this transaction so that rather than the Critical Section being the
+ // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
+ logger.trace("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
+ ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
+
+ final Response confirmTransactionResponse = Response.read(dis);
+ logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
+
+ switch (confirmTransactionResponse.getCode()) {
+ case CONFIRM_TRANSACTION:
+ break;
+ case BAD_CHECKSUM:
+ session.rollback();
+ throw new IOException(this + " Received a BadChecksum response from peer " + peer);
+ default:
+ throw new ProtocolException(this + " Received unexpected Response from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
+ }
+
+ // Commit the session so that we have persisted the data
+ session.commit();
+
+ if ( context.getAvailableRelationships().isEmpty() ) {
+ // Confirm that we received the data and the peer can now discard it but that the peer should not
+ // send any more data for a bit
+ logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
+ ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos);
+ } else {
+ // Confirm that we received the data and the peer can now discard it
+ logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
+ ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
+ }
+
+ stopWatch.stop();
+ final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
+ final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
+ final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+ final String dataSize = FormatUtils.formatDataSize(bytesReceived);
+ logger.info("{} Successfully receveied {} ({}) from {} in {} milliseconds at a rate of {}", new Object[] {
+ this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+ }
+
+ @Override
+ public void transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+ if ( !readyForFileTransfer ) {
+ throw new IllegalStateException("Cannot transfer files; handshake resolution was " + handshakeResponse);
+ }
+
+ FlowFile flowFile = session.get();
+ if ( flowFile == null ) {
+ return;
+ }
+
+ logger.debug("{} Sending FlowFiles to {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ String userDn = commsSession.getUserDn();
+ if ( userDn == null ) {
+ userDn = "none";
+ }
+
+ // Indicate that we would like to have some data
+ RequestType.SEND_FLOWFILES.writeRequestType(dos);
+ dos.flush();
+
+ final StopWatch stopWatch = new StopWatch(true);
+ final CRC32 crc = new CRC32();
+
+ long bytesSent = 0L;
+ final Set<FlowFile> flowFilesSent = new HashSet<>();
+ boolean continueTransaction = true;
+ String calculatedCRC = "";
+ final long startSendingNanos = System.nanoTime();
+ while (continueTransaction) {
+ final OutputStream flowFileOutputStream = useCompression ? new CompressionOutputStream(dos) : dos;
+ logger.debug("{} Sending {} to {}", this, flowFile, peer);
+
+ final CheckedOutputStream checkedOutStream = new CheckedOutputStream(flowFileOutputStream, crc);
+
+ final long startNanos = System.nanoTime();
+ flowFile = codec.encode(flowFile, session, checkedOutStream);
+ final long transferNanos = System.nanoTime() - startNanos;
+ final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+
+ // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
+ // Otherwise, do NOT close it because we don't want to close the underlying stream
+ // (CompressionOutputStream will not close the underlying stream when it's closed)
+ if ( useCompression ) {
+ checkedOutStream.close();
+ }
+
+ flowFilesSent.add(flowFile);
+ bytesSent += flowFile.getSize();
+ logger.debug("{} Sent {} to {}", this, flowFile, peer);
+
+ final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
+ session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transferMillis, false);
+ session.remove(flowFile);
+
+ final long sendingNanos = System.nanoTime() - startSendingNanos;
+ if ( sendingNanos < BATCH_SEND_NANOS ) {
+ flowFile = session.get();
+ } else {
+ flowFile = null;
+ }
+
+ continueTransaction = (flowFile != null);
+ if ( continueTransaction ) {
+ logger.debug("{} Sent CONTINUE_TRANSACTION indicator to {}", this, peer);
+ ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
+ } else {
+ logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
+ ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
+
+ calculatedCRC = String.valueOf( checkedOutStream.getChecksum().getValue() );
+ }
+ }
+
+ // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
+ final Response transactionConfirmationResponse = Response.read(dis);
+ if ( transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION ) {
+ // Confirm checksum and echo back the confirmation.
+ logger.trace("{} Received {} from {}", this, transactionConfirmationResponse, peer);
+ final String receivedCRC = transactionConfirmationResponse.getMessage();
+
+ if ( versionNegotiator.getVersion() > 3 ) {
+ if ( !receivedCRC.equals(calculatedCRC) ) {
+ ResponseCode.BAD_CHECKSUM.writeResponse(dos);
+ session.rollback();
+ throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as " + calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC + "; canceling transaction and rolling back session");
+ }
+ }
+
+ ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
+ } else {
+ throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
+ }
+
+ final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
+
+ final Response transactionResponse;
+ try {
+ transactionResponse = Response.read(dis);
+ } catch (final IOException e) {
+ logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator." +
+ " It is unknown whether or not the peer successfully received/processed the data." +
+ " Therefore, {} will be rolled back, possibly resulting in data duplication of {}",
+ this, peer, session, flowFileDescription);
+ session.rollback();
+ throw e;
+ }
+
+ logger.debug("{} Received {} from {}", this, transactionResponse, peer);
+ if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
+ peer.penalize(destination.getYieldPeriod(TimeUnit.MILLISECONDS));
+ } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
+ throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
+ }
+
+ // consume input stream entirely, ignoring its contents. If we
+ // don't do this, the Connection will not be returned to the pool
+ stopWatch.stop();
+ final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
+ final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+ final String dataSize = FormatUtils.formatDataSize(bytesSent);
+
+ session.commit();
+
+ logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
+ this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+ }
+
+ @Override
+ public VersionNegotiator getVersionNegotiator() {
+ return versionNegotiator;
+ }
+
+ @Override
+ public void shutdown(final Peer peer) throws IOException {
+ readyForFileTransfer = false;
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ logger.debug("{} Shutting down with {}", this, peer);
+ // Indicate that we would like to have some data
+ RequestType.SHUTDOWN.writeRequestType(dos);
+ dos.flush();
+ }
+
+ @Override
+ public String getResourceName() {
+ return "SocketFlowFileProtocol";
+ }
+
+ @Override
+ public String toString() {
+ return "SocketClientProtocol[CommsID=" + commsIdentifier + "]";
+ }
+}
[5/5] incubator-nifi git commit: NIFI-282: Begin refactoring and
creating client
Posted by ma...@apache.org.
NIFI-282: Begin refactoring and creating client
Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/fdf75846
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/fdf75846
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/fdf75846
Branch: refs/heads/site-to-site-client
Commit: fdf75846002877f7f0c857ff2b18593c6f2d825d
Parents: f21b502
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Jan 18 19:22:27 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Jan 18 19:22:27 2015 -0500
----------------------------------------------------------------------
nifi/commons/pom.xml | 9 +-
nifi/commons/site-to-site-client/pom.xml | 31 +
.../remote/AbstractCommunicationsSession.java | 54 ++
.../main/java/org/apache/nifi/remote/Peer.java | 113 ++++
.../java/org/apache/nifi/remote/PeerStatus.java | 72 +++
.../nifi/remote/PortAuthorizationResult.java | 25 +
.../nifi/remote/RemoteAuthorizationState.java | 27 +
.../nifi/remote/RemoteResourceInitiator.java | 64 ++
.../apache/nifi/remote/TransferDirection.java | 23 +
.../nifi/remote/VersionedRemoteResource.java | 24 +
.../apache/nifi/remote/client/DataPacket.java | 28 +
.../nifi/remote/client/SiteToSiteClient.java | 27 +
.../client/socket/EndpointConnectionState.java | 54 ++
.../socket/EndpointConnectionStatePool.java | 648 +++++++++++++++++++
.../nifi/remote/client/socket/SocketClient.java | 37 ++
.../remote/cluster/AdaptedNodeInformation.java | 66 ++
.../remote/cluster/ClusterNodeInformation.java | 67 ++
.../nifi/remote/cluster/NodeInformant.java | 22 +
.../nifi/remote/cluster/NodeInformation.java | 98 +++
.../remote/cluster/NodeInformationAdapter.java | 41 ++
.../apache/nifi/remote/codec/FlowFileCodec.java | 79 +++
.../remote/codec/StandardFlowFileCodec.java | 169 +++++
.../remote/exception/BadRequestException.java | 30 +
.../remote/exception/HandshakeException.java | 30 +
.../exception/NotAuthorizedException.java | 26 +
.../exception/PortNotRunningException.java | 26 +
.../remote/exception/ProtocolException.java | 34 +
.../exception/RequestExpiredException.java | 26 +
.../remote/exception/UnknownPortException.java | 26 +
.../exception/UnsupportedCodecException.java | 31 +
.../SocketChannelCommunicationsSession.java | 90 +++
.../remote/io/socket/SocketChannelInput.java | 66 ++
.../remote/io/socket/SocketChannelOutput.java | 58 ++
.../SSLSocketChannelCommunicationsSession.java | 93 +++
.../io/socket/ssl/SSLSocketChannelInput.java | 50 ++
.../io/socket/ssl/SSLSocketChannelOutput.java | 44 ++
.../nifi/remote/protocol/ClientProtocol.java | 78 +++
.../remote/protocol/CommunicationsInput.java | 27 +
.../remote/protocol/CommunicationsOutput.java | 27 +
.../remote/protocol/CommunicationsSession.java | 64 ++
.../nifi/remote/protocol/RequestType.java | 43 ++
.../protocol/socket/HandshakeProperty.java | 23 +
.../nifi/remote/protocol/socket/Response.java | 51 ++
.../remote/protocol/socket/ResponseCode.java | 152 +++++
.../protocol/socket/SocketClientProtocol.java | 517 +++++++++++++++
.../nifi/remote/util/PeerStatusCache.java | 43 ++
.../nifi/cluster/manager/ClusterManager.java | 2 +-
.../cluster/manager/impl/WebClusterManager.java | 5 +-
.../framework-bundle/framework/core-api/pom.xml | 5 +
.../nifi/cluster/AdaptedNodeInformation.java | 66 --
.../nifi/cluster/ClusterNodeInformation.java | 67 --
.../org/apache/nifi/cluster/NodeInformant.java | 22 -
.../apache/nifi/cluster/NodeInformation.java | 98 ---
.../nifi/cluster/NodeInformationAdapter.java | 39 --
.../apache/nifi/groups/RemoteProcessGroup.java | 6 -
.../main/java/org/apache/nifi/remote/Peer.java | 107 ---
.../java/org/apache/nifi/remote/PeerStatus.java | 72 ---
.../nifi/remote/PortAuthorizationResult.java | 25 -
.../nifi/remote/RemoteAuthorizationState.java | 27 -
.../org/apache/nifi/remote/RemoteGroupPort.java | 22 +-
.../apache/nifi/remote/TransferDirection.java | 23 -
.../nifi/remote/VersionedRemoteResource.java | 24 -
.../apache/nifi/remote/codec/FlowFileCodec.java | 79 ---
.../remote/exception/BadRequestException.java | 30 -
.../remote/exception/HandshakeException.java | 30 -
.../exception/NotAuthorizedException.java | 26 -
.../exception/PortNotRunningException.java | 26 -
.../remote/exception/ProtocolException.java | 34 -
.../exception/RequestExpiredException.java | 26 -
.../remote/exception/UnknownPortException.java | 26 -
.../nifi/remote/protocol/ClientProtocol.java | 78 ---
.../remote/protocol/CommunicationsInput.java | 27 -
.../remote/protocol/CommunicationsOutput.java | 27 -
.../remote/protocol/CommunicationsSession.java | 64 --
.../nifi/remote/protocol/RequestType.java | 43 --
.../nifi/remote/protocol/ServerProtocol.java | 2 +-
.../nifi/remote/StandardRemoteProcessGroup.java | 245 +------
.../framework/site-to-site/pom.xml | 4 +
.../remote/AbstractCommunicationsSession.java | 54 --
.../nifi/remote/RemoteResourceFactory.java | 42 +-
.../nifi/remote/SocketRemoteSiteListener.java | 2 +-
.../nifi/remote/StandardRemoteGroupPort.java | 498 +++-----------
.../remote/codec/StandardFlowFileCodec.java | 169 -----
.../exception/UnsupportedCodecException.java | 31 -
.../SocketChannelCommunicationsSession.java | 90 ---
.../remote/io/socket/SocketChannelInput.java | 66 --
.../remote/io/socket/SocketChannelOutput.java | 58 --
.../SSLSocketChannelCommunicationsSession.java | 93 ---
.../io/socket/ssl/SSLSocketChannelInput.java | 50 --
.../io/socket/ssl/SSLSocketChannelOutput.java | 44 --
.../socket/ClusterManagerServerProtocol.java | 7 +-
.../protocol/socket/HandshakeProperty.java | 23 -
.../nifi/remote/protocol/socket/Response.java | 51 --
.../remote/protocol/socket/ResponseCode.java | 152 -----
.../protocol/socket/SocketClientProtocol.java | 510 ---------------
.../socket/SocketFlowFileServerProtocol.java | 3 +-
.../remote/TestStandardRemoteGroupPort.java | 6 +-
nifi/nar-bundles/framework-bundle/pom.xml | 5 +
.../apache/nifi/remote/RemoteDestination.java | 37 ++
99 files changed, 3606 insertions(+), 3195 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/commons/pom.xml b/nifi/commons/pom.xml
index f85e337..43dc0d8 100644
--- a/nifi/commons/pom.xml
+++ b/nifi/commons/pom.xml
@@ -12,9 +12,7 @@
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+--><project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
@@ -39,6 +37,7 @@
<module>nifi-utils</module>
<module>nifi-web-utils</module>
<module>processor-utilities</module>
+ <module>site-to-site-client</module>
<module>wali</module>
- </modules>
-</project>
+ </modules>
+</project>
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/pom.xml b/nifi/commons/site-to-site-client/pom.xml
new file mode 100644
index 0000000..7719d55
--- /dev/null
+++ b/nifi/commons/site-to-site-client/pom.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0"?>
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-commons-parent</artifactId>
+ <version>0.0.1-incubating-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>site-to-site-client</artifactId>
+ <name>NiFi Site-to-Site Client</name>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-utils</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
new file mode 100644
index 0000000..4babb92
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+import org.apache.nifi.remote.protocol.CommunicationsSession;
+
+public abstract class AbstractCommunicationsSession implements CommunicationsSession {
+ private String userDn;
+
+ private volatile String uri;
+
+ public AbstractCommunicationsSession(final String uri) {
+ this.uri = uri;
+ }
+
+ @Override
+ public String toString() {
+ return uri;
+ }
+
+ @Override
+ public void setUri(final String uri) {
+ this.uri = uri;
+ }
+
+ @Override
+ public String getUri() {
+ return uri;
+ }
+
+ @Override
+ public String getUserDn() {
+ return userDn;
+ }
+
+ @Override
+ public void setUserDn(final String dn) {
+ this.userDn = dn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
new file mode 100644
index 0000000..e811c68
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.nifi.remote.protocol.CommunicationsSession;
+
+public class Peer {
+
+ private final CommunicationsSession commsSession;
+ private final String url;
+ private final String clusterUrl;
+ private final String host;
+ private long penalizationExpiration = 0L;
+ private boolean closed = false;
+
+ public Peer(final CommunicationsSession commsSession, final String peerUrl, final String clusterUrl) {
+ this.commsSession = commsSession;
+ this.url = peerUrl;
+ this.clusterUrl = clusterUrl;
+
+ try {
+ this.host = new URI(peerUrl).getHost();
+ } catch (final Exception e) {
+ throw new IllegalArgumentException("Invalid URL: " + peerUrl);
+ }
+ }
+
+ public String getUrl() {
+ return url;
+ }
+
+ public String getClusterUrl() {
+ return clusterUrl;
+ }
+
+ public CommunicationsSession getCommunicationsSession() {
+ return commsSession;
+ }
+
+ public void close() throws IOException {
+ this.closed = true;
+
+ // TODO: Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer
+ commsSession.close();
+ }
+
+ public void penalize(final long millis) {
+ penalizationExpiration = Math.max(penalizationExpiration, System.currentTimeMillis() + millis);
+ }
+
+ public boolean isPenalized() {
+ return penalizationExpiration > System.currentTimeMillis();
+ }
+
+ public boolean isClosed() {
+ return closed;
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ @Override
+ public int hashCode() {
+ return 8320 + url.hashCode();
+ }
+
+ @Override
+ public boolean equals(final Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof Peer)) {
+ return false;
+ }
+
+ final Peer other = (Peer) obj;
+ return this.url.equals(other.url);
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder();
+ sb.append("Peer[url=").append(url);
+ if (closed) {
+ sb.append(",CLOSED");
+ } else if (isPenalized()) {
+ sb.append(",PENALIZED");
+ }
+ sb.append("]");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
new file mode 100644
index 0000000..d1cb076
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+public class PeerStatus {
+
+ private final String hostname;
+ private final int port;
+ private final boolean secure;
+ private final int numFlowFiles;
+
+ public PeerStatus(final String hostname, final int port, final boolean secure, final int numFlowFiles) {
+ this.hostname = hostname;
+ this.port = port;
+ this.secure = secure;
+ this.numFlowFiles = numFlowFiles;
+ }
+
+ public String getHostname() {
+ return hostname;
+ }
+
+ public int getPort() {
+ return port;
+ }
+
+ public boolean isSecure() {
+ return secure;
+ }
+
+ public int getFlowFileCount() {
+ return numFlowFiles;
+ }
+
+ @Override
+ public String toString() {
+ return "PeerStatus[hostname=" + hostname + ",port=" + port + ",secure=" + secure + ",flowFileCount=" + numFlowFiles + "]";
+ }
+
+ @Override
+ public int hashCode() {
+ return 9824372 + hostname.hashCode() + port;
+ }
+
+ @Override
+ public boolean equals(final Object obj) {
+ if (obj == null) {
+ return false;
+ }
+
+ if (!(obj instanceof PeerStatus)) {
+ return false;
+ }
+
+ final PeerStatus other = (PeerStatus) obj;
+ return port == other.port && hostname.equals(other.hostname);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
new file mode 100644
index 0000000..8f2603a
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/PortAuthorizationResult.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+public interface PortAuthorizationResult {
+
+ boolean isAuthorized();
+
+ String getExplanation();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
new file mode 100644
index 0000000..12a3d33
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteAuthorizationState.java
@@ -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.remote;
+
+/**
+ *
+ */
+public enum RemoteAuthorizationState {
+
+ UNKNOWN,
+ UNAUTHORIZED,
+ AUTHORIZED;
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
new file mode 100644
index 0000000..8eb5d8d
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.nifi.remote.exception.HandshakeException;
+
+public class RemoteResourceInitiator {
+ public static final int RESOURCE_OK = 20;
+ public static final int DIFFERENT_RESOURCE_VERSION = 21;
+ public static final int ABORT = 255;
+
+
+ public static VersionedRemoteResource initiateResourceNegotiation(final VersionedRemoteResource resource, final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException {
+ // Write the classname of the RemoteStreamCodec, followed by its version
+ dos.writeUTF(resource.getResourceName());
+ final VersionNegotiator negotiator = resource.getVersionNegotiator();
+ dos.writeInt(negotiator.getVersion());
+ dos.flush();
+
+ // wait for response from server.
+ final int statusCode = dis.read();
+ switch (statusCode) {
+ case RESOURCE_OK: // server accepted our proposal of codec name/version
+ return resource;
+ case DIFFERENT_RESOURCE_VERSION: // server accepted our proposal of codec name but not the version
+ // Get server's preferred version
+ final int newVersion = dis.readInt();
+
+ // Determine our new preferred version that is no greater than the server's preferred version.
+ final Integer newPreference = negotiator.getPreferredVersion(newVersion);
+ // If we could not agree with server on a version, fail now.
+ if ( newPreference == null ) {
+ throw new HandshakeException("Could not agree on version for " + resource);
+ }
+
+ negotiator.setVersion(newPreference);
+
+ // Attempt negotiation of resource based on our new preferred version.
+ return initiateResourceNegotiation(resource, dis, dos);
+ case ABORT:
+ throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF());
+ default:
+ return null; // Unable to negotiate codec
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java
new file mode 100644
index 0000000..56432d5
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+public enum TransferDirection {
+
+ SEND,
+ RECEIVE;
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
new file mode 100644
index 0000000..bfccd98
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+public interface VersionedRemoteResource {
+
+ VersionNegotiator getVersionNegotiator();
+
+ String getResourceName();
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/DataPacket.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/DataPacket.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/DataPacket.java
new file mode 100644
index 0000000..ec77f2c
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/DataPacket.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.client;
+
+import java.io.InputStream;
+import java.util.Map;
+
+public interface DataPacket {
+
+ Map<String, String> getAttributes();
+
+ InputStream getData();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
new file mode 100644
index 0000000..47a09be
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -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.remote.client;
+
+import java.io.IOException;
+
+public interface SiteToSiteClient {
+
+ void send(DataPacket dataPacket) throws IOException;
+
+ DataPacket receive() throws IOException;
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java
new file mode 100644
index 0000000..f4ac727
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.client.socket;
+
+import org.apache.nifi.remote.Peer;
+import org.apache.nifi.remote.codec.FlowFileCodec;
+import org.apache.nifi.remote.protocol.socket.SocketClientProtocol;
+
+public class EndpointConnectionState {
+ private final Peer peer;
+ private final SocketClientProtocol socketClientProtocol;
+ private final FlowFileCodec codec;
+ private volatile long lastUsed;
+
+ public EndpointConnectionState(final Peer peer, final SocketClientProtocol socketClientProtocol, final FlowFileCodec codec) {
+ this.peer = peer;
+ this.socketClientProtocol = socketClientProtocol;
+ this.codec = codec;
+ }
+
+ public FlowFileCodec getCodec() {
+ return codec;
+ }
+
+ public SocketClientProtocol getSocketClientProtocol() {
+ return socketClientProtocol;
+ }
+
+ public Peer getPeer() {
+ return peer;
+ }
+
+ public void setLastTimeUsed() {
+ lastUsed = System.currentTimeMillis();
+ }
+
+ public long getLastTimeUsed() {
+ return lastUsed;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
new file mode 100644
index 0000000..2dd489d
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
@@ -0,0 +1,648 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.client.socket;
+
+import java.io.BufferedReader;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.channels.SocketChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+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.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+
+import javax.net.ssl.SSLContext;
+import javax.security.cert.CertificateExpiredException;
+import javax.security.cert.CertificateNotYetValidException;
+
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.remote.Peer;
+import org.apache.nifi.remote.PeerStatus;
+import org.apache.nifi.remote.RemoteDestination;
+import org.apache.nifi.remote.RemoteResourceInitiator;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.remote.cluster.NodeInformation;
+import org.apache.nifi.remote.codec.FlowFileCodec;
+import org.apache.nifi.remote.exception.BadRequestException;
+import org.apache.nifi.remote.exception.HandshakeException;
+import org.apache.nifi.remote.exception.PortNotRunningException;
+import org.apache.nifi.remote.exception.ProtocolException;
+import org.apache.nifi.remote.exception.TransmissionDisabledException;
+import org.apache.nifi.remote.exception.UnknownPortException;
+import org.apache.nifi.remote.io.socket.SocketChannelCommunicationsSession;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel;
+import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelCommunicationsSession;
+import org.apache.nifi.remote.protocol.CommunicationsSession;
+import org.apache.nifi.remote.protocol.socket.SocketClientProtocol;
+import org.apache.nifi.remote.util.PeerStatusCache;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class EndpointConnectionStatePool {
+ public static final long PEER_REFRESH_PERIOD = 60000L;
+ public static final String CATEGORY = "Site-to-Site";
+ private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
+
+ private static final Logger logger = LoggerFactory.getLogger(EndpointConnectionStatePool.class);
+
+ private final ConcurrentMap<String, BlockingQueue<EndpointConnectionState>> endpointConnectionMap = new ConcurrentHashMap<>();
+ private final ConcurrentMap<PeerStatus, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
+
+ private final AtomicLong peerIndex = new AtomicLong(0L);
+
+ private final ReentrantLock peerRefreshLock = new ReentrantLock();
+ private volatile List<PeerStatus> peerStatuses;
+ private volatile long peerRefreshTime = 0L;
+ private volatile PeerStatusCache peerStatusCache;
+ private final Set<CommunicationsSession> activeCommsChannels = new HashSet<>();
+
+ private final File peersFile;
+ private final EventReporter eventReporter;
+ private final SSLContext sslContext;
+
+ public EndpointConnectionStatePool(final EventReporter eventReporter, final File persistenceFile) {
+ this(null, eventReporter, persistenceFile);
+ }
+
+ public EndpointConnectionStatePool(final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) {
+ this.sslContext = sslContext;
+ this.peersFile = persistenceFile;
+ this.eventReporter = eventReporter;
+
+ Set<PeerStatus> recoveredStatuses;
+ if ( persistenceFile != null && persistenceFile.exists() ) {
+ try {
+ recoveredStatuses = recoverPersistedPeerStatuses(peersFile);
+ this.peerStatusCache = new PeerStatusCache(recoveredStatuses, peersFile.lastModified());
+ } catch (final IOException ioe) {
+ logger.warn("Failed to recover peer statuses from {} due to {}; will continue without loading information from file", persistenceFile, ioe);
+ }
+ } else {
+ peerStatusCache = null;
+ }
+ }
+
+ public EndpointConnectionState getEndpointConnectionState(final String clusterUrl, final RemoteDestination remoteDestination, final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
+ //
+ // Attempt to get a connection state that already exists for this URL.
+ //
+ BlockingQueue<EndpointConnectionState> connectionStateQueue = endpointConnectionMap.get(clusterUrl);
+ if ( connectionStateQueue == null ) {
+ connectionStateQueue = new LinkedBlockingQueue<>();
+ BlockingQueue<EndpointConnectionState> existingQueue = endpointConnectionMap.putIfAbsent(clusterUrl, connectionStateQueue);
+ if ( existingQueue != null ) {
+ connectionStateQueue = existingQueue;
+ }
+ }
+
+ FlowFileCodec codec = null;
+ CommunicationsSession commsSession = null;
+ SocketClientProtocol protocol = null;
+ EndpointConnectionState connectionState;
+ Peer peer = null;
+
+ do {
+ final PeerStatus peerStatus = getNextPeerStatus(direction);
+ if ( peerStatus == null ) {
+ return null;
+ }
+
+ connectionState = connectionStateQueue.poll();
+ logger.debug("{} Connection State for {} = {}", this, clusterUrl, connectionState);
+
+ // if we can't get an existing ConnectionState, create one
+ if ( connectionState == null ) {
+ protocol = new SocketClientProtocol();
+ protocol.setDestination(remoteDestination);
+
+ try {
+ commsSession = establishSiteToSiteConnection(peerStatus);
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ try {
+ RemoteResourceInitiator.initiateResourceNegotiation(protocol, dis, dos);
+ } catch (final HandshakeException e) {
+ try {
+ commsSession.close();
+ } catch (final IOException ioe) {
+ throw e;
+ }
+ }
+ } catch (final IOException e) {
+ }
+
+
+ final String peerUrl = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort();
+ peer = new Peer(commsSession, peerUrl, clusterUrl);
+
+ // perform handshake
+ try {
+ protocol.handshake(peer);
+
+ // handle error cases
+ if ( protocol.isDestinationFull() ) {
+ logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer);
+ penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+ connectionStateQueue.offer(connectionState);
+ continue;
+ } else if ( protocol.isPortInvalid() ) {
+ penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+ cleanup(protocol, peer);
+ throw new PortNotRunningException(peer.toString() + " indicates that port " + remoteDestination.getIdentifier() + " is not running");
+ } else if ( protocol.isPortUnknown() ) {
+ penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+ cleanup(protocol, peer);
+ throw new UnknownPortException(peer.toString() + " indicates that port " + remoteDestination.getIdentifier() + " is not known");
+ }
+
+ // negotiate the FlowFileCodec to use
+ codec = protocol.negotiateCodec(peer);
+ } catch (final PortNotRunningException | UnknownPortException e) {
+ throw e;
+ } catch (final Exception e) {
+ penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+ cleanup(protocol, peer);
+
+ final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? clusterUrl : peer, e.toString());
+ logger.error(message);
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", e);
+ }
+ throw e;
+ }
+
+ connectionState = new EndpointConnectionState(peer, protocol, codec);
+ } else {
+ final long lastTimeUsed = connectionState.getLastTimeUsed();
+ final long millisSinceLastUse = System.currentTimeMillis() - lastTimeUsed;
+ final long timeoutMillis = remoteDestination.getCommunicationsTimeout(TimeUnit.MILLISECONDS);
+
+ if ( timeoutMillis > 0L && millisSinceLastUse >= timeoutMillis ) {
+ cleanup(connectionState.getSocketClientProtocol(), connectionState.getPeer());
+ connectionState = null;
+ } else {
+ codec = connectionState.getCodec();
+ peer = connectionState.getPeer();
+ commsSession = peer.getCommunicationsSession();
+ protocol = connectionState.getSocketClientProtocol();
+ }
+ }
+ } while ( connectionState == null || codec == null || commsSession == null || protocol == null );
+
+ return connectionState;
+ }
+
+
+ public boolean offer(final EndpointConnectionState endpointConnectionState) {
+ final Peer peer = endpointConnectionState.getPeer();
+ if ( peer == null ) {
+ return false;
+ }
+
+ final String url = peer.getUrl();
+ if ( url == null ) {
+ return false;
+ }
+
+ final BlockingQueue<EndpointConnectionState> queue = endpointConnectionMap.get(url);
+ if ( queue == null ) {
+ return false;
+ }
+
+ return queue.offer(endpointConnectionState);
+ }
+
+ /**
+ * Updates internal state map to penalize a PeerStatus that points to the specified peer
+ * @param peer
+ */
+ public void penalize(final Peer peer, final long penalizationMillis) {
+ String host;
+ int port;
+ try {
+ final URI uri = new URI(peer.getUrl());
+ host = uri.getHost();
+ port = uri.getPort();
+ } catch (final URISyntaxException e) {
+ host = peer.getHost();
+ port = -1;
+ }
+
+ final PeerStatus status = new PeerStatus(host, port, true, 1);
+ Long expiration = peerTimeoutExpirations.get(status);
+ if ( expiration == null ) {
+ expiration = Long.valueOf(0L);
+ }
+
+ final long newExpiration = Math.max(expiration, System.currentTimeMillis() + penalizationMillis);
+ peerTimeoutExpirations.put(status, Long.valueOf(newExpiration));
+ }
+
+ private void cleanup(final SocketClientProtocol protocol, final Peer peer) {
+ if ( protocol != null && peer != null ) {
+ try {
+ protocol.shutdown(peer);
+ } catch (final TransmissionDisabledException e) {
+ // User disabled transmission.... do nothing.
+ logger.debug(this + " Transmission Disabled by User");
+ } catch (IOException e1) {
+ }
+ }
+
+ if ( peer != null ) {
+ try {
+ peer.close();
+ } catch (final TransmissionDisabledException e) {
+ // User disabled transmission.... do nothing.
+ logger.debug(this + " Transmission Disabled by User");
+ } catch (IOException e1) {
+ }
+ }
+ }
+
+ private PeerStatus getNextPeerStatus(final TransferDirection direction) {
+ List<PeerStatus> peerList = peerStatuses;
+ if ( (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD) && peerRefreshLock.tryLock() ) {
+ try {
+ try {
+ peerList = createPeerStatusList(direction);
+ } catch (final Exception e) {
+ final String message = String.format("%s Failed to update list of peers due to %s", this, e.toString());
+ logger.warn(message);
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", e);
+ }
+
+ eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
+ }
+
+ this.peerStatuses = peerList;
+ peerRefreshTime = System.currentTimeMillis();
+ } finally {
+ peerRefreshLock.unlock();
+ }
+ }
+
+ if ( peerList == null || peerList.isEmpty() ) {
+ return null;
+ }
+
+ PeerStatus peerStatus;
+ for (int i=0; i < peerList.size(); i++) {
+ final long idx = peerIndex.getAndIncrement();
+ final int listIndex = (int) (idx % peerList.size());
+ peerStatus = peerList.get(listIndex);
+
+ if ( isPenalized(peerStatus) ) {
+ logger.debug("{} {} is penalized; will not communicate with this peer", this, peerStatus);
+ } else {
+ return peerStatus;
+ }
+ }
+
+ logger.debug("{} All peers appear to be penalized; returning null", this);
+ return null;
+ }
+
+ private boolean isPenalized(final PeerStatus peerStatus) {
+ final Long expirationEnd = peerTimeoutExpirations.get(peerStatus);
+ return (expirationEnd == null ? false : expirationEnd > System.currentTimeMillis() );
+ }
+
+ private List<PeerStatus> createPeerStatusList(final TransferDirection direction) throws IOException, BadRequestException, HandshakeException, UnknownPortException, PortNotRunningException {
+ final Set<PeerStatus> statuses = getPeerStatuses();
+ if ( statuses == null ) {
+ return new ArrayList<>();
+ }
+
+ final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
+ final List<NodeInformation> nodeInfos = new ArrayList<>();
+ for ( final PeerStatus peerStatus : statuses ) {
+ final NodeInformation nodeInfo = new NodeInformation(peerStatus.getHostname(), peerStatus.getPort(), 0, peerStatus.isSecure(), peerStatus.getFlowFileCount());
+ nodeInfos.add(nodeInfo);
+ }
+ clusterNodeInfo.setNodeInformation(nodeInfos);
+ return formulateDestinationList(clusterNodeInfo, direction);
+ }
+
+
+ public Set<PeerStatus> getPeerStatuses() {
+ final PeerStatusCache cache = this.peerStatusCache;
+ if (cache == null || cache.getStatuses() == null || cache.getStatuses().isEmpty()) {
+ return null;
+ }
+
+ if (cache.getTimestamp() + PEER_CACHE_MILLIS < System.currentTimeMillis()) {
+ final Set<PeerStatus> equalizedSet = new HashSet<>(cache.getStatuses().size());
+ for (final PeerStatus status : cache.getStatuses()) {
+ final PeerStatus equalizedStatus = new PeerStatus(status.getHostname(), status.getPort(), status.isSecure(), 1);
+ equalizedSet.add(equalizedStatus);
+ }
+
+ return equalizedSet;
+ }
+
+ return cache.getStatuses();
+ }
+
+ private Set<PeerStatus> fetchRemotePeerStatuses(final URI destinationUri, final boolean secure) throws IOException, HandshakeException, UnknownPortException, PortNotRunningException, BadRequestException {
+ final String hostname = destinationUri.getHost();
+ final int port = destinationUri.getPort();
+
+ final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port, secure);
+ final Peer peer = new Peer(commsSession, "nifi://" + hostname + ":" + port, destinationUri.toString());
+ final SocketClientProtocol clientProtocol = new SocketClientProtocol();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ try {
+ RemoteResourceInitiator.initiateResourceNegotiation(clientProtocol, dis, dos);
+ } catch (final HandshakeException e) {
+ throw new BadRequestException(e.toString());
+ }
+
+ // TODO: Make the 30000 millis configurable
+ clientProtocol.handshake(peer, null, 30000);
+ final Set<PeerStatus> peerStatuses = clientProtocol.getPeerStatuses(peer);
+ persistPeerStatuses(peerStatuses);
+
+ try {
+ clientProtocol.shutdown(peer);
+ } catch (final IOException e) {
+ final String message = String.format("%s Failed to shutdown protocol when updating list of peers due to %s", this, e.toString());
+ logger.warn(message);
+ if (logger.isDebugEnabled()) {
+ logger.warn("", e);
+ }
+ }
+
+ try {
+ peer.close();
+ } catch (final IOException e) {
+ final String message = String.format("%s Failed to close resources when updating list of peers due to %s", this, e.toString());
+ logger.warn(message);
+ if (logger.isDebugEnabled()) {
+ logger.warn("", e);
+ }
+ }
+
+ return peerStatuses;
+ }
+
+
+ private void persistPeerStatuses(final Set<PeerStatus> statuses) {
+ if ( peersFile == null ) {
+ return;
+ }
+
+ try (final OutputStream fos = new FileOutputStream(peersFile);
+ final OutputStream out = new BufferedOutputStream(fos)) {
+
+ for (final PeerStatus status : statuses) {
+ final String line = status.getHostname() + ":" + status.getPort() + ":" + status.isSecure() + "\n";
+ out.write(line.getBytes(StandardCharsets.UTF_8));
+ }
+
+ } catch (final IOException e) {
+ logger.error("Failed to persist list of Peers due to {}; if restarted and peer's NCM is down, may be unable to transfer data until communications with NCM are restored", e.toString(), e);
+ }
+ }
+
+ private Set<PeerStatus> recoverPersistedPeerStatuses(final File file) throws IOException {
+ if (!file.exists()) {
+ return null;
+ }
+
+ final Set<PeerStatus> statuses = new HashSet<>();
+ try (final InputStream fis = new FileInputStream(file);
+ final BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
+
+ String line;
+ while ((line = reader.readLine()) != null) {
+ final String[] splits = line.split(Pattern.quote(":"));
+ if (splits.length != 3) {
+ continue;
+ }
+
+ final String hostname = splits[0];
+ final int port = Integer.parseInt(splits[1]);
+ final boolean secure = Boolean.parseBoolean(splits[2]);
+
+ statuses.add(new PeerStatus(hostname, port, secure, 1));
+ }
+ }
+
+ return statuses;
+ }
+
+
+ public CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException {
+ return establishSiteToSiteConnection(peerStatus.getHostname(), peerStatus.getPort(), peerStatus.isSecure());
+ }
+
+ public CommunicationsSession establishSiteToSiteConnection(final String hostname, final int port, final boolean secure) throws IOException {
+ final String destinationUri = "nifi://" + hostname + ":" + port;
+
+ CommunicationsSession commsSession = null;
+ try {
+ if ( secure ) {
+ if ( sslContext == null ) {
+ throw new IOException("Unable to communicate with " + hostname + ":" + port + " because it requires Secure Site-to-Site communications, but this instance is not configured for secure communications");
+ }
+
+ final SSLSocketChannel socketChannel = new SSLSocketChannel(sslContext, hostname, port, true);
+ socketChannel.connect();
+
+ commsSession = new SSLSocketChannelCommunicationsSession(socketChannel, destinationUri);
+
+ try {
+ commsSession.setUserDn(socketChannel.getDn());
+ } catch (final CertificateNotYetValidException | CertificateExpiredException ex) {
+ throw new IOException(ex);
+ }
+ } else {
+ final SocketChannel socketChannel = SocketChannel.open(new InetSocketAddress(hostname, port));
+ commsSession = new SocketChannelCommunicationsSession(socketChannel, destinationUri);
+ }
+
+ commsSession.getOutput().getOutputStream().write(CommunicationsSession.MAGIC_BYTES);
+
+ commsSession.setUri(destinationUri);
+ } catch (final IOException ioe) {
+ if ( commsSession != null ) {
+ commsSession.close();
+ }
+
+ throw ioe;
+ }
+
+ return commsSession;
+ }
+
+
+// private List<PeerStatus> formulateDestinationList(final ClusterNodeInformation clusterNodeInfo) throws IOException {
+// return formulateDestinationList(clusterNodeInfo, getConnectableType());
+// }
+
+ static List<PeerStatus> formulateDestinationList(final ClusterNodeInformation clusterNodeInfo, final TransferDirection direction) {
+ final Collection<NodeInformation> nodeInfoSet = clusterNodeInfo.getNodeInformation();
+ final int numDestinations = Math.max(128, nodeInfoSet.size());
+ final Map<NodeInformation, Integer> entryCountMap = new HashMap<>();
+
+ long totalFlowFileCount = 0L;
+ for (final NodeInformation nodeInfo : nodeInfoSet) {
+ totalFlowFileCount += nodeInfo.getTotalFlowFiles();
+ }
+
+ int totalEntries = 0;
+ for (final NodeInformation nodeInfo : nodeInfoSet) {
+ final int flowFileCount = nodeInfo.getTotalFlowFiles();
+ // don't allow any node to get more than 80% of the data
+ final double percentageOfFlowFiles = Math.min(0.8D, ((double) flowFileCount / (double) totalFlowFileCount));
+ final double relativeWeighting = (direction == TransferDirection.RECEIVE) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles;
+ final int entries = Math.max(1, (int) (numDestinations * relativeWeighting));
+
+ entryCountMap.put(nodeInfo, Math.max(1, entries));
+ totalEntries += entries;
+ }
+
+ final List<PeerStatus> destinations = new ArrayList<>(totalEntries);
+ for (int i=0; i < totalEntries; i++) {
+ destinations.add(null);
+ }
+ for ( final Map.Entry<NodeInformation, Integer> entry : entryCountMap.entrySet() ) {
+ final NodeInformation nodeInfo = entry.getKey();
+ final int numEntries = entry.getValue();
+
+ int skipIndex = numEntries;
+ for (int i=0; i < numEntries; i++) {
+ int n = (skipIndex * i);
+ while (true) {
+ final int index = n % destinations.size();
+ PeerStatus status = destinations.get(index);
+ if ( status == null ) {
+ status = new PeerStatus(nodeInfo.getHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure(), nodeInfo.getTotalFlowFiles());
+ destinations.set(index, status);
+ break;
+ } else {
+ n++;
+ }
+ }
+ }
+ }
+
+ final StringBuilder distributionDescription = new StringBuilder();
+ distributionDescription.append("New Weighted Distribution of Nodes:");
+ for ( final Map.Entry<NodeInformation, Integer> entry : entryCountMap.entrySet() ) {
+ final double percentage = entry.getValue() * 100D / (double) destinations.size();
+ distributionDescription.append("\n").append(entry.getKey()).append(" will receive ").append(percentage).append("% of FlowFiles");
+ }
+ logger.info(distributionDescription.toString());
+
+ // Jumble the list of destinations.
+ return destinations;
+ }
+
+
+ public void cleanupExpiredSockets() {
+ final List<EndpointConnectionState> states = new ArrayList<>();
+
+ for ( final BlockingQueue<EndpointConnectionState> queue : endpointConnectionMap.values() ) {
+ states.clear();
+
+ EndpointConnectionState state;
+ while ((state = queue.poll()) != null) {
+ // If the socket has not been used in 10 seconds, shut it down.
+ final long lastUsed = state.getLastTimeUsed();
+ if ( lastUsed < System.currentTimeMillis() - 10000L ) {
+ try {
+ state.getSocketClientProtocol().shutdown(state.getPeer());
+ } catch (final Exception e) {
+ logger.debug("Failed to shut down {} using {} due to {}",
+ new Object[] {state.getSocketClientProtocol(), state.getPeer(), e} );
+ }
+
+ cleanup(state.getSocketClientProtocol(), state.getPeer());
+ } else {
+ states.add(state);
+ }
+ }
+
+ queue.addAll(states);
+ }
+ }
+
+ public void shutdown() {
+ peerTimeoutExpirations.clear();
+
+ for ( final CommunicationsSession commsSession : activeCommsChannels ) {
+ commsSession.interrupt();
+ }
+
+ for ( final BlockingQueue<EndpointConnectionState> queue : endpointConnectionMap.values() ) {
+ EndpointConnectionState state;
+ while ( (state = queue.poll()) != null) {
+ cleanup(state.getSocketClientProtocol(), state.getPeer());
+ }
+ }
+
+ endpointConnectionMap.clear();
+ }
+
+ public void refreshPeers(final URI targetUri, final boolean secure) {
+ final PeerStatusCache existingCache = peerStatusCache;
+ if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
+ return;
+ }
+
+ try {
+ final Set<PeerStatus> statuses = fetchRemotePeerStatuses(targetUri, secure);
+ peerStatusCache = new PeerStatusCache(statuses);
+ logger.info("{} Successfully refreshed Peer Status; remote instance consists of {} peers", this, statuses.size());
+ } catch (Exception e) {
+ logger.warn("{} Unable to refresh Remote Group's peers due to {}", this, e);
+ if (logger.isDebugEnabled()) {
+ logger.warn("", e);
+ }
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
new file mode 100644
index 0000000..48e9cc5
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -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.remote.client.socket;
+
+import java.io.IOException;
+
+import org.apache.nifi.remote.client.DataPacket;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+
+public class SocketClient implements SiteToSiteClient {
+
+ @Override
+ public void send(final DataPacket dataPacket) throws IOException {
+ // TODO Auto-generated method stub
+ }
+
+ @Override
+ public DataPacket receive() throws IOException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java
new file mode 100644
index 0000000..6ca5812
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java
@@ -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.remote.cluster;
+
+public class AdaptedNodeInformation {
+
+ private String hostname;
+ private Integer siteToSitePort;
+ private int apiPort;
+ private boolean isSiteToSiteSecure;
+ private int totalFlowFiles;
+
+ public String getHostname() {
+ return hostname;
+ }
+
+ public void setHostname(String hostname) {
+ this.hostname = hostname;
+ }
+
+ public Integer getSiteToSitePort() {
+ return siteToSitePort;
+ }
+
+ public void setSiteToSitePort(Integer siteToSitePort) {
+ this.siteToSitePort = siteToSitePort;
+ }
+
+ public int getApiPort() {
+ return apiPort;
+ }
+
+ public void setApiPort(int apiPort) {
+ this.apiPort = apiPort;
+ }
+
+ public boolean isSiteToSiteSecure() {
+ return isSiteToSiteSecure;
+ }
+
+ public void setSiteToSiteSecure(boolean isSiteToSiteSecure) {
+ this.isSiteToSiteSecure = isSiteToSiteSecure;
+ }
+
+ public int getTotalFlowFiles() {
+ return totalFlowFiles;
+ }
+
+ public void setTotalFlowFiles(int totalFlowFiles) {
+ this.totalFlowFiles = totalFlowFiles;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java
new file mode 100644
index 0000000..1bc83b9
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.cluster;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collection;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+@XmlRootElement
+public class ClusterNodeInformation {
+
+ private Collection<NodeInformation> nodeInfo;
+
+ private static final JAXBContext JAXB_CONTEXT;
+
+ static {
+ try {
+ JAXB_CONTEXT = JAXBContext.newInstance(ClusterNodeInformation.class);
+ } catch (JAXBException e) {
+ throw new RuntimeException("Unable to create JAXBContext.", e);
+ }
+ }
+
+ public ClusterNodeInformation() {
+ this.nodeInfo = null;
+ }
+
+ public void setNodeInformation(final Collection<NodeInformation> nodeInfo) {
+ this.nodeInfo = nodeInfo;
+ }
+
+ @XmlJavaTypeAdapter(NodeInformationAdapter.class)
+ public Collection<NodeInformation> getNodeInformation() {
+ return nodeInfo;
+ }
+
+ public void marshal(final OutputStream os) throws JAXBException {
+ final Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
+ marshaller.marshal(this, os);
+ }
+
+ public static ClusterNodeInformation unmarshal(final InputStream is) throws JAXBException {
+ final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
+ return (ClusterNodeInformation) unmarshaller.unmarshal(is);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java
new file mode 100644
index 0000000..e46ff5c
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.cluster;
+
+public interface NodeInformant {
+
+ ClusterNodeInformation getNodeInformation();
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java
new file mode 100644
index 0000000..2041268
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java
@@ -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.remote.cluster;
+
+public class NodeInformation {
+
+ private final String hostname;
+ private final Integer siteToSitePort;
+ private final int apiPort;
+ private final boolean isSiteToSiteSecure;
+ private final int totalFlowFiles;
+
+ public NodeInformation(final String hostname, final Integer siteToSitePort, final int apiPort,
+ final boolean isSiteToSiteSecure, final int totalFlowFiles) {
+ this.hostname = hostname;
+ this.siteToSitePort = siteToSitePort;
+ this.apiPort = apiPort;
+ this.isSiteToSiteSecure = isSiteToSiteSecure;
+ this.totalFlowFiles = totalFlowFiles;
+ }
+
+ public String getHostname() {
+ return hostname;
+ }
+
+ public int getAPIPort() {
+ return apiPort;
+ }
+
+ public Integer getSiteToSitePort() {
+ return siteToSitePort;
+ }
+
+ public boolean isSiteToSiteSecure() {
+ return isSiteToSiteSecure;
+ }
+
+ public int getTotalFlowFiles() {
+ return totalFlowFiles;
+ }
+
+ @Override
+ public boolean equals(final Object obj) {
+ if (obj == this) {
+ return true;
+ }
+ if (obj == null) {
+ return false;
+ }
+ if (!(obj instanceof NodeInformation)) {
+ return false;
+ }
+
+ final NodeInformation other = (NodeInformation) obj;
+ if (!hostname.equals(other.hostname)) {
+ return false;
+ }
+ if (siteToSitePort == null && other.siteToSitePort != null) {
+ return false;
+ }
+ if (siteToSitePort != null && other.siteToSitePort == null) {
+ return false;
+ } else if (siteToSitePort != null && siteToSitePort.intValue() != other.siteToSitePort.intValue()) {
+ return false;
+ }
+ if (apiPort != other.apiPort) {
+ return false;
+ }
+ if (isSiteToSiteSecure != other.isSiteToSiteSecure) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 83832 + hostname.hashCode() + (siteToSitePort == null ? 8 : siteToSitePort.hashCode()) + apiPort + (isSiteToSiteSecure ? 3829 : 0);
+ }
+
+ @Override
+ public String toString() {
+ return "Node[" + hostname + ":" + apiPort + "]";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java
new file mode 100644
index 0000000..440463c
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.cluster;
+
+import javax.xml.bind.annotation.adapters.XmlAdapter;
+
+import org.apache.nifi.remote.cluster.NodeInformation;
+
+public class NodeInformationAdapter extends XmlAdapter<AdaptedNodeInformation, NodeInformation> {
+
+ @Override
+ public NodeInformation unmarshal(final AdaptedNodeInformation adapted) throws Exception {
+ return new NodeInformation(adapted.getHostname(), adapted.getSiteToSitePort(), adapted.getApiPort(), adapted.isSiteToSiteSecure(), adapted.getTotalFlowFiles());
+ }
+
+ @Override
+ public AdaptedNodeInformation marshal(final NodeInformation nodeInformation) throws Exception {
+ final AdaptedNodeInformation adapted = new AdaptedNodeInformation();
+ adapted.setHostname(nodeInformation.getHostname());
+ adapted.setSiteToSitePort(nodeInformation.getSiteToSitePort());
+ adapted.setApiPort(nodeInformation.getAPIPort());
+ adapted.setSiteToSiteSecure(nodeInformation.isSiteToSiteSecure());
+ adapted.setTotalFlowFiles(nodeInformation.getTotalFlowFiles());
+ return adapted;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/fdf75846/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
new file mode 100644
index 0000000..b4206b3
--- /dev/null
+++ b/nifi/commons/site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.codec;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.remote.VersionedRemoteResource;
+import org.apache.nifi.remote.exception.ProtocolException;
+import org.apache.nifi.remote.exception.TransmissionDisabledException;
+
+/**
+ * <p>
+ * Provides a mechanism for encoding and decoding FlowFiles as streams so that
+ * they can be transferred remotely.
+ * </p>
+ */
+public interface FlowFileCodec extends VersionedRemoteResource {
+
+ /**
+ * Returns a List of all versions that this codec is able to support, in the
+ * order that they are preferred by the codec
+ *
+ * @return
+ */
+ public List<Integer> getSupportedVersions();
+
+ /**
+ * Encodes a FlowFile and its content as a single stream of data and writes
+ * that stream to the output. If checksum is not null, it will be calculated
+ * as the stream is read
+ *
+ * @param flowFile the FlowFile to encode
+ * @param session a session that can be used to transactionally create and
+ * transfer flow files
+ * @param outStream the stream to write the data to
+ *
+ * @return the updated FlowFile
+ *
+ * @throws IOException
+ */
+ FlowFile encode(FlowFile flowFile, ProcessSession session, OutputStream outStream) throws IOException, TransmissionDisabledException;
+
+ /**
+ * Decodes the contents of the InputStream, interpreting the data to
+ * determine the next FlowFile's attributes and content, as well as their
+ * destinations. If not null, checksum will be used to calculate the
+ * checksum as the data is read.
+ *
+ * @param stream an InputStream containing FlowFiles' contents, attributes,
+ * and destinations
+ * @param session
+ *
+ * @return the FlowFile that was created, or <code>null</code> if the stream
+ * was out of data
+ *
+ * @throws IOException
+ * @throws ProtocolException if the input is malformed
+ */
+ FlowFile decode(InputStream stream, ProcessSession session) throws IOException, ProtocolException, TransmissionDisabledException;
+}