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;
+}