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/02/23 20:56:54 UTC

[01/29] incubator-nifi git commit: NIFI-282: Bug fixes; documentation improvements; removed code marked as 'FOR TESTING'

Repository: incubator-nifi
Updated Branches:
  refs/heads/develop 42f69196c -> 705ee852b


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
deleted file mode 100644
index d18a4ee..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
deleted file mode 100644
index 0822b6a..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
deleted file mode 100644
index 9e451fd..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
deleted file mode 100644
index 26c0164..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java
index a526f4c..391d52b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
deleted file mode 100644
index c4519cd..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
deleted file mode 100644
index eae1940..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
deleted file mode 100644
index 0e588cd..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
deleted file mode 100644
index d4b4f61..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index 5edd4f9..eb22b0e 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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;
@@ -41,24 +40,27 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.remote.Peer;
 import org.apache.nifi.remote.PortAuthorizationResult;
 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;
 import org.apache.nifi.remote.io.CompressionInputStream;
 import org.apache.nifi.remote.io.CompressionOutputStream;
 import org.apache.nifi.remote.protocol.CommunicationsSession;
+import org.apache.nifi.remote.protocol.DataPacket;
 import org.apache.nifi.remote.protocol.RequestType;
 import org.apache.nifi.remote.protocol.ServerProtocol;
+import org.apache.nifi.remote.util.StandardDataPacket;
 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;
 
@@ -76,10 +78,14 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
     private FlowFileCodec negotiatedFlowFileCodec = null;
     private String transitUriPrefix = null;
     
-    private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1);
+    private int requestedBatchCount = 0;
+    private long requestedBatchBytes = 0L;
+    private long requestedBatchNanos = 0L;
+    private static final long DEFAULT_BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L);
+    
+    private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1);
     private final Logger logger = LoggerFactory.getLogger(SocketFlowFileServerProtocol.class);
     
-    private static final long BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
 
     
     @Override
@@ -135,68 +141,90 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
                 throw new HandshakeException("Received unknown property: " + propertyName);
             }
             
-            switch (property) {
-                case GZIP: {
-                    useGzip = Boolean.parseBoolean(value);
-                    break;
-                }
-                case REQUEST_EXPIRATION_MILLIS:
-                    requestExpirationMillis = Long.parseLong(value);
-                    break;
-                case PORT_IDENTIFIER: {
-                    Port receivedPort = rootGroup.getInputPort(value);
-                    if ( receivedPort == null ) {
-                        receivedPort = rootGroup.getOutputPort(value);
-                    }
-                    if ( receivedPort == null ) {
-                        logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
-                        ResponseCode.UNKNOWN_PORT.writeResponse(dos);
-                        throw new HandshakeException("Received unknown port identifier: " + value);
-                    }
-                    if ( !(receivedPort instanceof RootGroupPort) ) {
-                        logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
-                        ResponseCode.UNKNOWN_PORT.writeResponse(dos);
-                        throw new HandshakeException("Received port identifier " + value + ", but this Port is not a RootGroupPort");
-                    }
-                    
-                    this.port = (RootGroupPort) receivedPort;
-                    final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn());
-                    if ( !portAuthResult.isAuthorized() ) {
-                        logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation());
-                        ResponseCode.UNAUTHORIZED.writeResponse(dos, portAuthResult.getExplanation());
-                        responseWritten = true;
+            try {
+                switch (property) {
+                    case GZIP: {
+                        useGzip = Boolean.parseBoolean(value);
                         break;
                     }
-                    
-                    if ( !receivedPort.isValid() ) {
-                        logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
-                        ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port is not valid");
-                        responseWritten = true;
+                    case REQUEST_EXPIRATION_MILLIS:
+                        requestExpirationMillis = Long.parseLong(value);
                         break;
-                    }
-                    
-                    if ( !receivedPort.isRunning() ) {
-                        logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
-                        ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port not running");
-                        responseWritten = true;
+                    case BATCH_COUNT:
+                        requestedBatchCount = Integer.parseInt(value);
+                        if ( requestedBatchCount < 0 ) {
+                            throw new HandshakeException("Cannot request Batch Count less than 1; requested value: " + value);
+                        }
                         break;
-                    }
-                    
-                    // PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this
-                    // we we will simply not service the request but the sender will timeout
-                    if ( getVersionNegotiator().getVersion() > 1 ) {
-                        for ( final Connection connection : port.getConnections() ) {
-                            if ( connection.getFlowFileQueue().isFull() ) {
-                                logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", receivedPort);
-                                ResponseCode.PORTS_DESTINATION_FULL.writeResponse(dos);
-                                responseWritten = true;
-                                break;
+                    case BATCH_SIZE:
+                        requestedBatchBytes = Long.parseLong(value);
+                        if ( requestedBatchBytes < 0 ) {
+                            throw new HandshakeException("Cannot request Batch Size less than 1; requested value: " + value);
+                        }
+                        break;
+                    case BATCH_DURATION:
+                        requestedBatchNanos = TimeUnit.MILLISECONDS.toNanos(Long.parseLong(value));
+                        if ( requestedBatchNanos < 0 ) {
+                            throw new HandshakeException("Cannot request Batch Duration less than 1; requested value: " + value);
+                        }
+                        break;
+                    case PORT_IDENTIFIER: {
+                        Port receivedPort = rootGroup.getInputPort(value);
+                        if ( receivedPort == null ) {
+                            receivedPort = rootGroup.getOutputPort(value);
+                        }
+                        if ( receivedPort == null ) {
+                            logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
+                            ResponseCode.UNKNOWN_PORT.writeResponse(dos);
+                            throw new HandshakeException("Received unknown port identifier: " + value);
+                        }
+                        if ( !(receivedPort instanceof RootGroupPort) ) {
+                            logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
+                            ResponseCode.UNKNOWN_PORT.writeResponse(dos);
+                            throw new HandshakeException("Received port identifier " + value + ", but this Port is not a RootGroupPort");
+                        }
+                        
+                        this.port = (RootGroupPort) receivedPort;
+                        final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn());
+                        if ( !portAuthResult.isAuthorized() ) {
+                            logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation());
+                            ResponseCode.UNAUTHORIZED.writeResponse(dos, portAuthResult.getExplanation());
+                            responseWritten = true;
+                            break;
+                        }
+                        
+                        if ( !receivedPort.isValid() ) {
+                            logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
+                            ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port is not valid");
+                            responseWritten = true;
+                            break;
+                        }
+                        
+                        if ( !receivedPort.isRunning() ) {
+                            logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
+                            ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port not running");
+                            responseWritten = true;
+                            break;
+                        }
+                        
+                        // PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this
+                        // we we will simply not service the request but the sender will timeout
+                        if ( getVersionNegotiator().getVersion() > 1 ) {
+                            for ( final Connection connection : port.getConnections() ) {
+                                if ( connection.getFlowFileQueue().isFull() ) {
+                                    logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", receivedPort);
+                                    ResponseCode.PORTS_DESTINATION_FULL.writeResponse(dos);
+                                    responseWritten = true;
+                                    break;
+                                }
                             }
                         }
+                        
+                        break;
                     }
-                    
-                    break;
                 }
+            } catch (final NumberFormatException nfe) {
+                throw new HandshakeException("Received invalid value for property '" + property + "'; invalid value: " + value);
             }
         }
         
@@ -205,11 +233,6 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
             ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.GZIP.name());
             throw new HandshakeException("Missing Property " + HandshakeProperty.GZIP.name());
         }
-        if ( port == null ) {
-            logger.debug("Responding with ResponseCode MISSING_PROPERTY because Port Identifier property is missing");
-            ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.PORT_IDENTIFIER.name());
-            throw new HandshakeException("Missing Property " + HandshakeProperty.PORT_IDENTIFIER.name());
-        }
         
         // send "OK" response
         if ( !responseWritten ) {
@@ -244,6 +267,10 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
         
+        if ( port == null ) {
+        	RemoteResourceFactory.rejectCodecNegotiation(dis, dos, "Cannot transfer FlowFiles because no port was specified");
+        }
+        
         // Negotiate the FlowFileCodec to use.
         try {
             negotiatedFlowFileCodec = RemoteResourceFactory.receiveCodecNegotiation(dis, dos);
@@ -306,7 +333,16 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
             final CheckedOutputStream checkedOutputStream = new CheckedOutputStream(flowFileOutputStream, crc);
 
             final StopWatch transferWatch = new StopWatch(true);
-            flowFile = codec.encode(flowFile, session, checkedOutputStream);
+            
+            final FlowFile toSend = flowFile;
+            session.read(flowFile, new InputStreamCallback() {
+				@Override
+				public void process(final InputStream in) throws IOException {
+					final DataPacket dataPacket = new StandardDataPacket(toSend.getAttributes(), in, toSend.getSize());
+					codec.encode(dataPacket, checkedOutputStream);
+				}
+            });
+            
             final long transmissionMillis = transferWatch.getElapsed(TimeUnit.MILLISECONDS);
             
             // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
@@ -323,8 +359,25 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
             session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transmissionMillis, false);
             session.remove(flowFile);
             
+            // determine if we should check for more data on queue.
             final long sendingNanos = System.nanoTime() - startNanos;
-            if ( sendingNanos < BATCH_NANOS ) { 
+            boolean poll = true;
+            if ( sendingNanos >= requestedBatchNanos && requestedBatchNanos > 0L ) {
+                poll = false;
+            }
+            if ( bytesSent >= requestedBatchBytes && requestedBatchBytes > 0L ) {
+                poll = false;
+            }
+            if ( flowFilesSent.size() >= requestedBatchCount && requestedBatchCount > 0 ) {
+                poll = false;
+            }
+            
+            if ( requestedBatchNanos == 0 && requestedBatchBytes == 0 && requestedBatchCount == 0 ) {
+                poll = (sendingNanos < DEFAULT_BATCH_NANOS);
+            }
+            
+            if ( poll ) { 
+                // we've not elapsed the requested sending duration, so get more data.
                 flowFile = session.get();
             } else {
                 flowFile = null;
@@ -429,7 +482,11 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
             final InputStream flowFileInputStream = useGzip ? new CompressionInputStream(dis) : dis;
             final CheckedInputStream checkedInputStream = new CheckedInputStream(flowFileInputStream, crc);
 
-            FlowFile flowFile = codec.decode(checkedInputStream, session);
+            final DataPacket dataPacket = codec.decode(checkedInputStream);
+            FlowFile flowFile = session.create();
+            flowFile = session.importFrom(dataPacket.getData(), flowFile);
+            flowFile = session.putAllAttributes(flowFile, dataPacket.getAttributes());
+            
             final long transferNanos = System.nanoTime() - startNanos;
             final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
             final String sourceSystemFlowFileUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
@@ -451,6 +508,10 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
                     continueTransaction = false;
                     calculatedCRC = String.valueOf(checkedInputStream.getChecksum().getValue());
                     break;
+                case CANCEL_TRANSACTION:
+                    logger.info("{} Received CancelTransaction indicator from {} with explanation {}", this, peer, transactionResponse.getMessage());
+                    session.rollback();
+                    return 0;
                 default:
                     throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionResponse);
             }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
deleted file mode 100644
index e074010..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
+++ /dev/null
@@ -1,97 +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.StandardRemoteGroupPort;
-import org.apache.nifi.remote.PeerStatus;
-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 {
-
-    @Test
-    public void testFormulateDestinationListForOutput() throws IOException {
-        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
-        final List<NodeInformation> collection = new ArrayList<>();
-        collection.add(new NodeInformation("ShouldGetMedium", 1, 1111, true, 4096));
-        collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 10240));
-        collection.add(new NodeInformation("ShouldGetLittle", 3, 3333, true, 1024));
-        collection.add(new NodeInformation("ShouldGetMedium", 4, 4444, true, 4096));
-        collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
-
-        clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_OUTPUT_PORT);
-        for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
-        }
-    }
-    
-    @Test
-    public void testFormulateDestinationListForOutputHugeDifference() throws IOException {
-        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
-        final List<NodeInformation> collection = new ArrayList<>();
-        collection.add(new NodeInformation("ShouldGetLittle", 1, 1111, true, 500));
-        collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000));
-
-        clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_OUTPUT_PORT);
-        for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
-        }
-    }
-    
-    
-    
-    
-    @Test
-    public void testFormulateDestinationListForInputPorts() throws IOException {
-        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
-        final List<NodeInformation> collection = new ArrayList<>();
-        collection.add(new NodeInformation("ShouldGetMedium", 1, 1111, true, 4096));
-        collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 10240));
-        collection.add(new NodeInformation("ShouldGetLots", 3, 3333, true, 1024));
-        collection.add(new NodeInformation("ShouldGetMedium", 4, 4444, true, 4096));
-        collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
-
-        clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_INPUT_PORT);
-        for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
-        }
-    }
-    
-    @Test
-    public void testFormulateDestinationListForInputPortsHugeDifference() throws IOException {
-        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
-        final List<NodeInformation> collection = new ArrayList<>();
-        collection.add(new NodeInformation("ShouldGetLots", 1, 1111, true, 500));
-        collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 50000));
-
-        clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = StandardRemoteGroupPort.formulateDestinationList(clusterNodeInfo, ConnectableType.REMOTE_INPUT_PORT);
-        for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 8eda682..eb395c9 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -624,6 +624,11 @@
                 <artifactId>nifi-utils</artifactId>
                 <version>0.0.2-incubating-SNAPSHOT</version>
             </dependency>
+			<dependency>
+				<groupId>org.apache.nifi</groupId>
+				<artifactId>nifi-site-to-site-client</artifactId>
+				<version>0.0.2-incubating-SNAPSHOT</version>
+			</dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-web-utils</artifactId>


[21/29] incubator-nifi git commit: NIFI-365: Initial implementation of spark receiver

Posted by ma...@apache.org.
NIFI-365: Initial implementation of spark receiver


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/8506a0ce
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/8506a0ce
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/8506a0ce

Branch: refs/heads/develop
Commit: 8506a0ce4ae71d2a103e11d6b8bf7e5832c81e09
Parents: 7ab4392
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 18 21:19:58 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 18 21:19:58 2015 -0500

----------------------------------------------------------------------
 nifi/nifi-external/README.md                    |  19 ++
 nifi/nifi-external/nifi-spark-receiver/pom.xml  |  38 ++++
 .../org/apache/nifi/spark/NiFiDataPacket.java   |  40 ++++
 .../org/apache/nifi/spark/NiFiReceiver.java     | 198 +++++++++++++++++++
 nifi/nifi-external/pom.xml                      |  29 +++
 nifi/pom.xml                                    |   1 +
 6 files changed, 325 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8506a0ce/nifi/nifi-external/README.md
----------------------------------------------------------------------
diff --git a/nifi/nifi-external/README.md b/nifi/nifi-external/README.md
new file mode 100644
index 0000000..649ad17
--- /dev/null
+++ b/nifi/nifi-external/README.md
@@ -0,0 +1,19 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+# nifi-external
+
+The nifi-external module is a location where components can be developed by the NiFi team
+that are not intended to be used directly by NiFi but are to be used within other frameworks
+in order to integrate with NiFi.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8506a0ce/nifi/nifi-external/nifi-spark-receiver/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-external/nifi-spark-receiver/pom.xml b/nifi/nifi-external/nifi-spark-receiver/pom.xml
new file mode 100644
index 0000000..b21d554
--- /dev/null
+++ b/nifi/nifi-external/nifi-spark-receiver/pom.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi</artifactId>
+        <version>0.0.2-incubating-SNAPSHOT</version>
+    </parent>
+    <groupId>org.apache.nifi</groupId>
+    <artifactId>nifi-spark-receiver</artifactId>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.spark</groupId>
+			<artifactId>spark-streaming_2.10</artifactId>
+			<version>1.2.0</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-site-to-site-client</artifactId>
+			<version>0.0.2-incubating-SNAPSHOT</version>
+		</dependency>
+	</dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8506a0ce/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiDataPacket.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiDataPacket.java b/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiDataPacket.java
new file mode 100644
index 0000000..2f08dc5
--- /dev/null
+++ b/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiDataPacket.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.spark;
+
+import java.util.Map;
+
+/**
+ * <p>
+ * The NiFiDataPacket provides a packaging around a NiFi FlowFile. It wraps both a FlowFile's
+ * content and its attributes so that they can be processed by Spark
+ * </p>
+ */
+public interface NiFiDataPacket {
+
+	/**
+	 * Returns the contents of a NiFi FlowFile
+	 * @return
+	 */
+	byte[] getContent();
+
+	/**
+	 * Returns a Map of attributes that are associated with the NiFi FlowFile
+	 * @return
+	 */
+	Map<String, String> getAttributes();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8506a0ce/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java b/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java
new file mode 100644
index 0000000..9f31062
--- /dev/null
+++ b/nifi/nifi-external/nifi-spark-receiver/src/main/java/org/apache/nifi/spark/NiFiReceiver.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.spark;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+import org.apache.nifi.remote.client.SiteToSiteClientConfig;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.receiver.Receiver;
+
+
+/**
+ * <p>
+ * The <code>NiFiReceiver</code> is a Reliable Receiver that provides a way to pull data 
+ * from Apache NiFi so that it can be processed by Spark Streaming. The NiFi Receiver connects 
+ * to NiFi instance provided in the config and requests data from
+ * the OutputPort that is named. In NiFi, when an OutputPort is added to the root process group,
+ * it acts as a queue of data for remote clients. This receiver is then able to pull that data
+ * from NiFi reliably.
+ * </p>
+ * 
+ * <p>
+ * It is important to note that if pulling data from a NiFi cluster, the URL that should be used
+ * is that of the NiFi Cluster Manager. The Receiver will automatically handle determining the nodes
+ * in that cluster and pull from those nodes as appropriate.
+ * </p>
+ * 
+ * <p>
+ * In order to use the NiFiReceiver, you will need to first build a {@link SiteToSiteClientConfig} to provide 
+ * to the constructor. This can be achieved by using the {@link SiteToSiteClient.Builder}.
+ * Below is an example snippet of driver code to pull data from NiFi that is running on localhost:8080. This
+ * example assumes that NiFi exposes and OutputPort on the root group named "Data For Spark".
+ * Additionally, it assumes that the data that it will receive from this OutputPort is text
+ * data, as it will map the byte array received from NiFi to a UTF-8 Encoded string.
+ * </p>
+ * 
+ * <code>
+ * <pre>
+ * Pattern SPACE = Pattern.compile(" ");
+ * 
+ * // Build a Site-to-site client config
+ * SiteToSiteClientConfig config = new SiteToSiteClient.Builder()
+ *   .setUrl("http://localhost:8080/nifi")
+ *   .setPortName("Data For Spark")
+ *   .buildConfig();
+ * 
+ * SparkConf sparkConf = new SparkConf().setAppName("NiFi-Spark Streaming example");
+ * JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000L));
+ * 
+ * // Create a JavaReceiverInputDStream using a NiFi receiver so that we can pull data from 
+ * // specified Port
+ * JavaReceiverInputDStream<NiFiDataPacket> packetStream = 
+ *     ssc.receiverStream(new NiFiReceiver(clientConfig, StorageLevel.MEMORY_ONLY()));
+ * 
+ * // Map the data from NiFi to text, ignoring the attributes
+ * JavaDStream<String> text = packetStream.map(new Function<NiFiDataPacket, String>() {
+ *   public String call(final NiFiDataPacket dataPacket) throws Exception {
+ *     return new String(dataPacket.getContent(), StandardCharsets.UTF_8);
+ *   }
+ * });
+ * 
+ * // Split the words by spaces
+ * JavaDStream<String> words = text.flatMap(new FlatMapFunction<String, String>() {
+ *   public Iterable<String> call(final String text) throws Exception {
+ *     return Arrays.asList(SPACE.split(text));
+ *   }
+ * });
+ * 	    
+ * // Map each word to the number 1, then aggregate by key
+ * JavaPairDStream<String, Integer> wordCounts = words.mapToPair(
+ *   new PairFunction<String, String, Integer>() {
+ *     public Tuple2<String, Integer> call(String s) {
+ *       return new Tuple2<String, Integer>(s, 1);
+ *     }
+ *   }).reduceByKey(new Function2<Integer, Integer, Integer>() {
+ *     public Integer call(Integer i1, Integer i2) {
+ *       return i1 + i2;
+ *     }
+ *    }
+ *  );
+ * 
+ * // print the results
+ * wordCounts.print();
+ * ssc.start();
+ * ssc.awaitTermination();
+ * </pre>
+ * </code>
+ */
+public class NiFiReceiver extends Receiver<NiFiDataPacket> {
+	private static final long serialVersionUID = 3067274587595578836L;
+	private final SiteToSiteClientConfig clientConfig;
+	
+	public NiFiReceiver(final SiteToSiteClientConfig clientConfig, final StorageLevel storageLevel) {
+		super(storageLevel);
+		this.clientConfig = clientConfig;
+	}
+	
+	@Override
+	public void onStart() {
+		final Thread thread = new Thread(new ReceiveRunnable());
+		thread.setDaemon(true);
+		thread.setName("NiFi Receiver");
+		thread.start();
+	}
+
+	@Override
+	public void onStop() {
+	}
+
+	class ReceiveRunnable implements Runnable {
+		public ReceiveRunnable() {
+		}
+		
+		public void run() {
+			try {
+				final SiteToSiteClient client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build();
+				try {
+					while ( !isStopped() ) {
+						final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
+						DataPacket dataPacket = transaction.receive();
+						if ( dataPacket == null ) {
+							transaction.confirm();
+							transaction.complete();
+							
+							// no data available. Wait a bit and try again
+							try {
+								Thread.sleep(1000L);
+							} catch (InterruptedException e) {}
+							
+							continue;
+						}
+	
+						final List<NiFiDataPacket> dataPackets = new ArrayList<NiFiDataPacket>();
+						do {
+							// Read the data into a byte array and wrap it along with the attributes
+							// into a NiFiDataPacket.
+							final InputStream inStream = dataPacket.getData();
+							final byte[] data = new byte[(int) dataPacket.getSize()];
+							StreamUtils.fillBuffer(inStream, data);
+							
+							final Map<String, String> attributes = dataPacket.getAttributes();
+							final NiFiDataPacket NiFiDataPacket = new NiFiDataPacket() {
+								public byte[] getContent() {
+									return data;
+								}
+
+								public Map<String, String> getAttributes() {
+									return attributes;
+								}
+							};
+							
+							dataPackets.add(NiFiDataPacket);
+							dataPacket = transaction.receive();
+						} while ( dataPacket != null );
+
+						// Confirm transaction to verify the data
+						transaction.confirm();
+						
+						store(dataPackets.iterator());
+						
+						transaction.complete();
+					}
+				} finally {
+					try {
+						client.close();
+					} catch (final IOException ioe) {
+						reportError("Failed to close client", ioe);
+					}
+				}
+			} catch (final IOException ioe) {
+				restart("Failed to receive data from NiFi", ioe);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8506a0ce/nifi/nifi-external/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-external/pom.xml b/nifi/nifi-external/pom.xml
new file mode 100644
index 0000000..878098f
--- /dev/null
+++ b/nifi/nifi-external/pom.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi</artifactId>
+        <version>0.0.2-incubating-SNAPSHOT</version>
+    </parent>
+    <groupId>org.apache.nifi</groupId>
+    <artifactId>nifi-external</artifactId>
+    <packaging>pom</packaging>
+    <modules>
+        <module>nifi-spark-receiver</module>
+    </modules>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8506a0ce/nifi/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 6297161..5881db7 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -65,6 +65,7 @@
         <module>nifi-assembly</module>
         <module>nifi-docs</module>
         <module>nifi-maven-archetypes</module>
+		<module>nifi-external</module>
     </modules>
     <scm>
         <connection>scm:git:git://git.apache.org/incubator-nifi.git</connection>


[11/29] incubator-nifi git commit: NIFI-282: - Updating the label on the menu item for refreshing a remote process group.

Posted by ma...@apache.org.
NIFI-282:
- Updating the label on the menu item for refreshing a remote process group.

Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/4ab5c308
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/4ab5c308
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/4ab5c308

Branch: refs/heads/develop
Commit: 4ab5c308fd7abcabaacea2457fd7867a1e1f9210
Parents: 4c04fe6
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Feb 10 07:28:12 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Feb 10 07:28:12 2015 -0500

----------------------------------------------------------------------
 .../nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4ab5c308/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
index dfb8d7a..81c6846 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
@@ -359,7 +359,7 @@ nf.ContextMenu = (function () {
         {condition: hasUpstream, menuItem: {img: 'images/iconSmallRelationship.png', text: 'Upstream connections', action: 'showUpstream'}},
         {condition: hasDownstream, menuItem: {img: 'images/iconSmallRelationship.png', text: 'Downstream connections', action: 'showDownstream'}},
         {condition: hasUsage, menuItem: {img: 'images/iconUsage.png', text: 'Usage', action: 'showUsage'}},
-        {condition: isRemoteProcessGroup, menuItem: {img: 'images/iconRefresh.png', text: 'Refresh flow', action: 'refreshRemoteFlow'}},
+        {condition: isRemoteProcessGroup, menuItem: {img: 'images/iconRefresh.png', text: 'Refresh', action: 'refreshRemoteFlow'}},
         {condition: isRemoteProcessGroup, menuItem: {img: 'images/iconGoTo.png', text: 'Go to', action: 'openUri'}},
         {condition: isColorable, menuItem: {img: 'images/iconColor.png', text: 'Change color', action: 'fillColor'}},
         {condition: isNotConnection, menuItem: {img: 'images/iconCenterView.png', text: 'Center in view', action: 'center'}},


[08/29] incubator-nifi git commit: NIFI-282: Refactoring to allow for separate client

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
deleted file mode 100644
index 8c23e28..0000000
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
+++ /dev/null
@@ -1,835 +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.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.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantLock;
-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 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.client.SiteToSiteClientConfig;
-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.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.remote.util.RemoteNiFiUtils;
-import org.apache.nifi.reporting.Severity;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.web.api.dto.ControllerDTO;
-import org.apache.nifi.web.api.dto.PortDTO;
-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";
-    public static final long REMOTE_REFRESH_MILLIS = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
-
-    private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
-
-	private static final Logger logger = LoggerFactory.getLogger(EndpointConnectionStatePool.class);
-	
-	private final BlockingQueue<EndpointConnectionState> connectionStateQueue = new LinkedBlockingQueue<>();
-    private final ConcurrentMap<PeerStatus, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
-    private final URI clusterUrl;
-    private final String apiUri;
-    
-    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;
-    private final ScheduledExecutorService taskExecutor;
-    
-    private final ReadWriteLock listeningPortRWLock = new ReentrantReadWriteLock();
-    private final Lock remoteInfoReadLock = listeningPortRWLock.readLock();
-    private final Lock remoteInfoWriteLock = listeningPortRWLock.writeLock();
-    private Integer siteToSitePort;
-    private Boolean siteToSiteSecure;
-    private long remoteRefreshTime;
-    private final Map<String, String> inputPortMap = new HashMap<>();	// map input port name to identifier
-    private final Map<String, String> outputPortMap = new HashMap<>();	// map output port name to identifier
-    
-    private volatile int commsTimeout;
-
-    public EndpointConnectionStatePool(final String clusterUrl, final int commsTimeoutMillis, final EventReporter eventReporter, final File persistenceFile) {
-    	this(clusterUrl, commsTimeoutMillis, null, eventReporter, persistenceFile);
-    }
-    
-    public EndpointConnectionStatePool(final String clusterUrl, final int commsTimeoutMillis, final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) {
-    	try {
-    		this.clusterUrl = new URI(clusterUrl);
-    	} catch (final URISyntaxException e) {
-    		throw new IllegalArgumentException("Invalid Cluster URL: " + clusterUrl);
-    	}
-    	
-    	// Trim the trailing /
-        String uriPath = this.clusterUrl.getPath();
-        if (uriPath.endsWith("/")) {
-            uriPath = uriPath.substring(0, uriPath.length() - 1);
-        }
-        apiUri = this.clusterUrl.getScheme() + "://" + this.clusterUrl.getHost() + ":" + this.clusterUrl.getPort() + uriPath + "-api";
-        
-    	this.sslContext = sslContext;
-    	this.peersFile = persistenceFile;
-    	this.eventReporter = eventReporter;
-    	this.commsTimeout = commsTimeoutMillis;
-    	
-    	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;
-    	}
-
-    	// Initialize a scheduled executor and run some maintenance tasks in the background to kill off old, unused
-    	// connections and keep our list of peers up-to-date.
-    	taskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
-    		private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
-    		
-			@Override
-			public Thread newThread(final Runnable r) {
-				final Thread thread = defaultFactory.newThread(r);
-				thread.setName("NiFi Site-to-Site Connection Pool Maintenance");
-				return thread;
-			}
-    	});
-
-    	taskExecutor.scheduleWithFixedDelay(new Runnable() {
-			@Override
-			public void run() {
-				refreshPeers();
-			}
-    	}, 0, 5, TimeUnit.SECONDS);
-
-    	taskExecutor.scheduleWithFixedDelay(new Runnable() {
-			@Override
-			public void run() {
-				cleanupExpiredSockets();
-			}
-    	}, 5, 5, TimeUnit.SECONDS);
-    }
-    
-    
-    public EndpointConnectionState getEndpointConnectionState(final RemoteDestination remoteDestination, final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
-        return getEndpointConnectionState(remoteDestination, direction, null);
-    }
-    
-    
-    
-    public EndpointConnectionState getEndpointConnectionState(final RemoteDestination remoteDestination, final TransferDirection direction, final SiteToSiteClientConfig config) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
-    	//
-        // Attempt to get a connection state that already exists for this URL.
-        //
-        FlowFileCodec codec = null;
-        CommunicationsSession commsSession = null;
-        SocketClientProtocol protocol = null;
-        EndpointConnectionState connectionState;
-        Peer peer = null;
-        
-        final List<EndpointConnectionState> addBack = new ArrayList<>();
-        try {
-            do {
-                final PeerStatus peerStatus = getNextPeerStatus(direction);
-                if ( peerStatus == null ) {
-                	return null;
-                }
-    
-                connectionState = connectionStateQueue.poll();
-                logger.debug("{} Connection State for {} = {}", this, clusterUrl, connectionState);
-                
-                if ( connectionState == null && !addBack.isEmpty() ) {
-                    // all available connections have been penalized.
-                    return null;
-                }
-                
-                if ( connectionState != null && connectionState.getPeer().isPenalized() ) {
-                    // we have a connection, but it's penalized. We want to add it back to the queue
-                    // when we've found one to use.
-                    addBack.add(connectionState);
-                    continue;
-                }
-                
-                // 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.toString());
-    
-                    // set properties based on config
-                    if ( config != null ) {
-                        protocol.setTimeout((int) config.getTimeout(TimeUnit.MILLISECONDS));
-                        protocol.setPreferredBatchCount(config.getPreferredBatchCount());
-                        protocol.setPreferredBatchSize(config.getPreferredBatchSize());
-                        protocol.setPreferredBatchDuration(config.getPreferredBatchDuration(TimeUnit.MILLISECONDS));
-                    }
-                    
-                    // 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;
-                    
-                    if ( commsTimeout > 0L && millisSinceLastUse >= commsTimeout ) {
-                        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 );
-        } finally {
-            if ( !addBack.isEmpty() ) {
-                connectionStateQueue.addAll(addBack);
-            }
-        }
-        
-        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;
-    	}
-    	
-    	return connectionStateQueue.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);
-                    }
-                    
-                    if ( eventReporter != null ) {
-                    	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, 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);
-    }
-    
-    
-    private 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() throws IOException, HandshakeException, UnknownPortException, PortNotRunningException {
-    	final String hostname = clusterUrl.getHost();
-        final int port = getSiteToSitePort();
-    	
-    	final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port);
-        final Peer peer = new Peer(commsSession, "nifi://" + hostname + ":" + port, clusterUrl.toString());
-        final SocketClientProtocol clientProtocol = new SocketClientProtocol();
-        final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
-        final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
-        RemoteResourceInitiator.initiateResourceNegotiation(clientProtocol, dis, dos);
-
-        clientProtocol.setTimeout(commsTimeout);
-        clientProtocol.handshake(peer, null);
-        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;
-    }
-    
-    
-    private CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException {
-    	return establishSiteToSiteConnection(peerStatus.getHostname(), peerStatus.getPort());
-    }
-    
-    private CommunicationsSession establishSiteToSiteConnection(final String hostname, final int port) throws IOException {
-    	if ( siteToSiteSecure == null ) {
-    		throw new IOException("Remote NiFi instance " + clusterUrl + " is not currently configured to accept site-to-site connections");
-    	}
-    	
-        final String destinationUri = "nifi://" + hostname + ":" + port;
-
-        CommunicationsSession commsSession = null;
-        try {
-	        if ( siteToSiteSecure ) {
-	            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;
-    }
-    
-    
-    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;
-    }
-    
-    
-    private void cleanupExpiredSockets() {
-        final List<EndpointConnectionState> states = new ArrayList<>();
-        
-        EndpointConnectionState state;
-        while ((state = connectionStateQueue.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);
-            }
-        }
-        
-        connectionStateQueue.addAll(states);
-    }
-    
-    public void shutdown() {
-    	taskExecutor.shutdown();
-    	peerTimeoutExpirations.clear();
-            
-        for ( final CommunicationsSession commsSession : activeCommsChannels ) {
-            commsSession.interrupt();
-        }
-        
-        EndpointConnectionState state;
-        while ( (state = connectionStateQueue.poll()) != null)  {
-            cleanup(state.getSocketClientProtocol(), state.getPeer());
-        }
-    }
-    
-    public void terminate(final EndpointConnectionState state) {
-        cleanup(state.getSocketClientProtocol(), state.getPeer());
-    }
-    
-    private void refreshPeers() {
-        final PeerStatusCache existingCache = peerStatusCache;
-        if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
-            return;
-        }
-
-        try {
-            final Set<PeerStatus> statuses = fetchRemotePeerStatuses();
-            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);
-            }
-        }
-    }
-    
-    
-    public String getInputPortIdentifier(final String portName) throws IOException {
-        return getPortIdentifier(portName, inputPortMap);
-    }
-    
-    public String getOutputPortIdentifier(final String portName) throws IOException {
-    	return getPortIdentifier(portName, outputPortMap);
-    }
-    
-    
-    private String getPortIdentifier(final String portName, final Map<String, String> portMap) throws IOException {
-    	String identifier;
-    	remoteInfoReadLock.lock();
-        try {
-        	identifier = portMap.get(portName);
-        } finally {
-        	remoteInfoReadLock.unlock();
-        }
-        
-        if ( identifier != null ) {
-        	return identifier;
-        }
-        
-        refreshRemoteInfo();
-
-    	remoteInfoReadLock.lock();
-        try {
-        	return portMap.get(portName);
-        } finally {
-        	remoteInfoReadLock.unlock();
-        }
-    }
-    
-    
-    private ControllerDTO refreshRemoteInfo() throws IOException {
-    	final boolean webInterfaceSecure = clusterUrl.toString().startsWith("https");
-        final RemoteNiFiUtils utils = new RemoteNiFiUtils(webInterfaceSecure ? sslContext : null);
-		final ControllerDTO controller = utils.getController(URI.create(apiUri + "/controller"), commsTimeout);
-        
-        remoteInfoWriteLock.lock();
-        try {
-            this.siteToSitePort = controller.getRemoteSiteListeningPort();
-            this.siteToSiteSecure = controller.isSiteToSiteSecure();
-            
-            inputPortMap.clear();
-            for (final PortDTO inputPort : controller.getInputPorts()) {
-            	inputPortMap.put(inputPort.getName(), inputPort.getId());
-            }
-            
-            outputPortMap.clear();
-            for ( final PortDTO outputPort : controller.getOutputPorts()) {
-            	outputPortMap.put(outputPort.getName(), outputPort.getId());
-            }
-            
-            this.remoteRefreshTime = System.currentTimeMillis();
-        } finally {
-        	remoteInfoWriteLock.unlock();
-        }
-        
-        return controller;
-    }
-    
-    /**
-     * @return the port that the remote instance is listening on for
-     * site-to-site communication, or <code>null</code> if the remote instance
-     * is not configured to allow site-to-site communications.
-     *
-     * @throws IOException if unable to communicate with the remote instance
-     */
-    private Integer getSiteToSitePort() throws IOException {
-        Integer listeningPort;
-        remoteInfoReadLock.lock();
-        try {
-            listeningPort = this.siteToSitePort;
-            if (listeningPort != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
-                return listeningPort;
-            }
-        } finally {
-        	remoteInfoReadLock.unlock();
-        }
-
-        final ControllerDTO controller = refreshRemoteInfo();
-        listeningPort = controller.getRemoteSiteListeningPort();
-
-        return listeningPort;
-    }
- 
-    /**
-     * Returns {@code true} if the remote instance is configured for secure site-to-site communications,
-     * {@code false} otherwise.
-     * 
-     * @return
-     * @throws IOException
-     */
-    public boolean isSecure() throws IOException {
-        remoteInfoReadLock.lock();
-        try {
-            final Boolean secure = this.siteToSiteSecure;
-            if (secure != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
-                return secure;
-            }
-        } finally {
-        	remoteInfoReadLock.unlock();
-        }
-
-        final ControllerDTO controller = refreshRemoteInfo();
-        return controller.isSiteToSiteSecure();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
index 0494d04..6fa934b 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -24,23 +24,23 @@ import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.SiteToSiteClient;
 import org.apache.nifi.remote.client.SiteToSiteClientConfig;
-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;
 import org.apache.nifi.remote.protocol.DataPacket;
 import org.apache.nifi.util.ObjectHolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class SocketClient implements SiteToSiteClient {
+    private static final Logger logger = LoggerFactory.getLogger(SocketClient.class);
+    
     private final SiteToSiteClientConfig config;
-	private final EndpointConnectionStatePool pool;
+	private final EndpointConnectionPool pool;
 	private final boolean compress;
 	private final String portName;
 	private final long penalizationNanos;
 	private volatile String portIdentifier;
 	
 	public SocketClient(final SiteToSiteClientConfig config) {
-		pool = new EndpointConnectionStatePool(config.getUrl(), (int) config.getTimeout(TimeUnit.MILLISECONDS), 
+		pool = new EndpointConnectionPool(config.getUrl(), (int) config.getTimeout(TimeUnit.MILLISECONDS), 
 				config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile());
 		
 		this.config = config;
@@ -66,44 +66,55 @@ public class SocketClient implements SiteToSiteClient {
 			return id;
 		}
 		
+		final String portId;
 		if ( direction == TransferDirection.SEND ) {
-			return pool.getInputPortIdentifier(this.portName);
+			portId = pool.getInputPortIdentifier(this.portName);
 		} else {
-			return pool.getOutputPortIdentifier(this.portName);
+			portId = pool.getOutputPortIdentifier(this.portName);
 		}
+		
+		if (portId == null) {
+		    logger.debug("Unable to resolve port [{}] to an identifier", portName);
+		} else {
+		    logger.debug("Resolved port [{}] to identifier [{}]", portName, portId);
+		}
+		
+		return portId;
 	}
 	
 	
+	private RemoteDestination createRemoteDestination(final String portId) {
+	    return new RemoteDestination() {
+            @Override
+            public String getIdentifier() {
+                return portId;
+            }
+
+            @Override
+            public long getYieldPeriod(final TimeUnit timeUnit) {
+                return timeUnit.convert(penalizationNanos, TimeUnit.NANOSECONDS);
+            }
+
+            @Override
+            public boolean isUseCompression() {
+                return compress;
+            }
+        };
+	}
+	
 	@Override
 	public Transaction createTransaction(final TransferDirection direction) throws IOException {
-		final String portId = getPortIdentifier(TransferDirection.SEND);
+		final String portId = getPortIdentifier(direction);
 		
 		if ( portId == null ) {
-			throw new IOException("Could not find Port with name " + portName + " for remote NiFi instance");
+			throw new IOException("Could not find Port with name '" + portName + "' for remote NiFi instance");
 		}
 		
-		final RemoteDestination remoteDestination = new RemoteDestination() {
-			@Override
-			public String getIdentifier() {
-				return portId;
-			}
-
-			@Override
-			public long getYieldPeriod(final TimeUnit timeUnit) {
-				return timeUnit.convert(penalizationNanos, TimeUnit.NANOSECONDS);
-			}
-
-			@Override
-			public boolean isUseCompression() {
-				return compress;
-			}
-		};
+		final RemoteDestination remoteDestination = createRemoteDestination(portId);
 		
-		final EndpointConnectionState connectionState;
-		try {
-			connectionState = pool.getEndpointConnectionState(remoteDestination, direction);
-		} catch (final ProtocolException | HandshakeException | PortNotRunningException | UnknownPortException e) {
-			throw new IOException(e);
+		final EndpointConnection connectionState = pool.getEndpointConnection(remoteDestination, direction, getConfig());
+		if ( connectionState == null ) {
+		    return null;
 		}
 		
 		final Transaction transaction = connectionState.getSocketClientProtocol().startTransaction(
@@ -111,7 +122,7 @@ public class SocketClient implements SiteToSiteClient {
 		
 		// Wrap the transaction in a new one that will return the EndpointConnectionState back to the pool whenever
 		// the transaction is either completed or canceled.
-		final ObjectHolder<EndpointConnectionState> connectionStateRef = new ObjectHolder<>(connectionState);
+		final ObjectHolder<EndpointConnection> connectionStateRef = new ObjectHolder<>(connectionState);
 		return new Transaction() {
 			@Override
 			public void confirm() throws IOException {
@@ -119,11 +130,16 @@ public class SocketClient implements SiteToSiteClient {
 			}
 
 			@Override
+			public void complete() throws IOException {
+			    complete(false);
+			}
+			
+			@Override
 			public void complete(final boolean requestBackoff) throws IOException {
 				try {
 					transaction.complete(requestBackoff);
 				} finally {
-				    final EndpointConnectionState state = connectionStateRef.get();
+				    final EndpointConnection state = connectionStateRef.get();
 				    if ( state != null ) {
 				        pool.offer(connectionState);
 				        connectionStateRef.set(null);
@@ -136,7 +152,7 @@ public class SocketClient implements SiteToSiteClient {
 				try {
 					transaction.cancel(explanation);
 				} finally {
-                    final EndpointConnectionState state = connectionStateRef.get();
+                    final EndpointConnection state = connectionStateRef.get();
                     if ( state != null ) {
                         pool.terminate(connectionState);
                         connectionStateRef.set(null);
@@ -149,7 +165,7 @@ public class SocketClient implements SiteToSiteClient {
 			    try {
 			        transaction.error();
 			    } finally {
-                    final EndpointConnectionState state = connectionStateRef.get();
+                    final EndpointConnection state = connectionStateRef.get();
                     if ( state != null ) {
                         pool.terminate(connectionState);
                         connectionStateRef.set(null);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
index b61fc65..d4d55e1 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
@@ -16,8 +16,15 @@
  */
 package org.apache.nifi.remote.exception;
 
-public class HandshakeException extends Exception {
+import java.io.IOException;
 
+
+/**
+ * A HandshakeException occurs when the client and the remote NiFi instance do not agree
+ * on some condition during the handshake. For example, if the NiFi instance does not recognize
+ * one of the parameters that the client passes during the Handshaking phase.
+ */
+public class HandshakeException extends IOException {
     private static final long serialVersionUID = 178192341908726L;
 
     public HandshakeException(final String message) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
index af0f467..8b97832 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
@@ -16,8 +16,12 @@
  */
 package org.apache.nifi.remote.exception;
 
-public class PortNotRunningException extends Exception {
-
+/**
+ * PortNotRunningException occurs when the remote NiFi instance reports
+ * that the Port that the client is attempting to communicate with is not
+ * currently running and therefore communications with that Port are not allowed.
+ */
+public class PortNotRunningException extends ProtocolException {
     private static final long serialVersionUID = -2790940982005516375L;
 
     public PortNotRunningException(final String message) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
index e12348a..45a4e15 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
@@ -18,6 +18,10 @@ package org.apache.nifi.remote.exception;
 
 import java.io.IOException;
 
+/**
+ * A ProtocolException occurs when unexpected data is received, for example
+ * an invalid Response Code.
+ */
 public class ProtocolException extends IOException {
 
     private static final long serialVersionUID = 5763900324505818495L;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
index e6a0fe7..592a1b3 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
@@ -16,8 +16,11 @@
  */
 package org.apache.nifi.remote.exception;
 
-public class UnknownPortException extends Exception {
-
+/**
+ * An UnknownPortException indicates that the remote NiFi instance has reported that
+ * the endpoint that the client attempted to communicate with does not exist.
+ */
+public class UnknownPortException extends ProtocolException {
     private static final long serialVersionUID = -2790940982005516375L;
 
     public UnknownPortException(final String message) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
index 9e451fd..7dffddd 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
@@ -63,4 +63,9 @@ public class SocketChannelInput implements CommunicationsInput {
     public void interrupt() {
         interruptableIn.interrupt();
     }
+
+    @Override
+    public void consume() throws IOException {
+        socketIn.consume();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
index 60ef33f..01fb9f2 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
@@ -47,4 +47,9 @@ public class SSLSocketChannelInput implements CommunicationsInput {
     public long getBytesRead() {
         return countingIn.getBytesRead();
     }
+
+    @Override
+    public void consume() throws IOException {
+        in.consume();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
index befbdaa..36a0e8d 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
@@ -40,9 +40,9 @@ public interface ClientProtocol extends VersionedRemoteResource {
 
     FlowFileCodec negotiateCodec(Peer peer) throws IOException, ProtocolException;
 
-    void receiveFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
+    int 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;
+    int transferFlowFiles(Peer peer, ProcessContext context, ProcessSession session, FlowFileCodec codec) throws IOException, ProtocolException;
 
     void shutdown(Peer peer) throws IOException, ProtocolException;
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
index d2e2946..5e56902 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
@@ -21,6 +21,12 @@ import java.io.InputStream;
 
 public interface CommunicationsInput {
 
+    /**
+     * Reads all data currently on the socket and throws it away
+     * @throws IOException
+     */
+    void consume() throws IOException;
+    
     InputStream getInputStream() throws IOException;
 
     long getBytesRead();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
index 5f194f8..e321663 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@ -150,6 +150,7 @@ public class SocketClientProtocol implements ClientProtocol {
             }
         }
         
+        logger.debug("Handshaking with properties {}", properties);
         dos.writeInt(properties.size());
         for ( final Map.Entry<HandshakeProperty, String> entry : properties.entrySet() ) {
             dos.writeUTF(entry.getKey().name());
@@ -269,13 +270,13 @@ public class SocketClientProtocol implements ClientProtocol {
             throw new IllegalStateException("Cannot start transaction; handshake resolution was " + handshakeResponse);
         }
         
-        return new SocketClientTransaction(versionNegotiator.getVersion(), peer, codec, 
+        return new SocketClientTransaction(versionNegotiator.getVersion(), destination.getIdentifier(), peer, codec, 
         		direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS));
     }
 
 
     @Override
-    public void receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+    public int receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
     	final String userDn = peer.getCommunicationsSession().getUserDn();
     	final Transaction transaction = startTransaction(peer, codec, TransferDirection.RECEIVE);
     	
@@ -288,7 +289,7 @@ public class SocketClientProtocol implements ClientProtocol {
     		final DataPacket dataPacket = transaction.receive();
     		if ( dataPacket == null ) {
     		    if ( flowFilesReceived.isEmpty() ) {
-    		        peer.penalize(destination.getYieldPeriod(TimeUnit.MILLISECONDS));
+    		        peer.penalize(destination.getIdentifier(), destination.getYieldPeriod(TimeUnit.MILLISECONDS));
     		    }
     			break;
     		}
@@ -322,25 +323,25 @@ public class SocketClientProtocol implements ClientProtocol {
 		transaction.complete(applyBackpressure);
 		logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
 
-		if ( flowFilesReceived.isEmpty() ) {
-		    return;
+		if ( !flowFilesReceived.isEmpty() ) {
+    		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 });
 		}
 		
-		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 });
+		return flowFilesReceived.size();
     }
 
     
     @Override
-    public void transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+    public int transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
 		FlowFile flowFile = session.get();
 		if (flowFile == null) {
-			return;
+			return 0;
 		}
 
 		try {
@@ -401,6 +402,8 @@ public class SocketClientProtocol implements ClientProtocol {
 	        final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
 	        logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
 	            this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+	        
+	        return flowFilesSent.size();
 		} catch (final Exception e) {
 			session.rollback();
 			throw e;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
index edb360e..cf8f9b2 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
@@ -19,6 +19,7 @@ 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.zip.CRC32;
 import java.util.zip.CheckedInputStream;
@@ -29,6 +30,8 @@ import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.codec.FlowFileCodec;
 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.DataPacket;
 import org.apache.nifi.remote.protocol.RequestType;
 import org.slf4j.Logger;
@@ -47,14 +50,16 @@ public class SocketClientTransaction implements Transaction {
 	private final boolean compress;
 	private final Peer peer;
 	private final int penaltyMillis;
+	private final String destinationId;
 	
 	private boolean dataAvailable = false;
 	private int transfers = 0;
 	private TransactionState state;
 	
-	SocketClientTransaction(final int protocolVersion, final Peer peer, final FlowFileCodec codec, 
+	SocketClientTransaction(final int protocolVersion, final String destinationId, final Peer peer, final FlowFileCodec codec, 
 			final TransferDirection direction, final boolean useCompression, final int penaltyMillis) throws IOException {
 		this.protocolVersion = protocolVersion;
+		this.destinationId = destinationId;
 		this.peer = peer;
 		this.codec = codec;
 		this.direction = direction;
@@ -140,7 +145,8 @@ public class SocketClientTransaction implements Transaction {
         	}
         	
             logger.debug("{} Receiving data from {}", this, peer);
-            final DataPacket packet = codec.decode(new CheckedInputStream(dis, crc));
+            final InputStream dataIn = compress ? new CompressionInputStream(dis) : dis;
+            final DataPacket packet = codec.decode(new CheckedInputStream(dataIn, crc));
             
             if ( packet == null ) {
                 this.dataAvailable = false;
@@ -174,7 +180,8 @@ public class SocketClientTransaction implements Transaction {
     
             logger.debug("{} Sending data to {}", this, peer);
     
-    		final OutputStream out = new CheckedOutputStream(dos, crc);
+            final OutputStream dataOut = compress ? new CompressionOutputStream(dos) : dos;
+    		final OutputStream out = new CheckedOutputStream(dataOut, crc);
             codec.encode(dataPacket, out);
             
             // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
@@ -208,6 +215,10 @@ public class SocketClientTransaction implements Transaction {
 		}
 	}
 	
+	@Override
+	public void complete() throws IOException {
+	    complete(false);
+	}
 	
 	@Override
 	public void complete(boolean requestBackoff) throws IOException {
@@ -246,7 +257,7 @@ public class SocketClientTransaction implements Transaction {
                 
                 logger.debug("{} Received {} from {}", this, transactionResponse, peer);
                 if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
-                    peer.penalize(penaltyMillis);
+                    peer.penalize(destinationId, penaltyMillis);
                 } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
                     throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
index d8899ea..275e40c 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
@@ -39,7 +39,7 @@ public class TestEndpointConnectionStatePool {
         collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
 
         clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
             System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
         }
@@ -53,7 +53,7 @@ public class TestEndpointConnectionStatePool {
         collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000));
 
         clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
             System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
         }
@@ -73,7 +73,7 @@ public class TestEndpointConnectionStatePool {
         collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
 
         clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
             System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
         }
@@ -87,7 +87,7 @@ public class TestEndpointConnectionStatePool {
         collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 50000));
 
         clusterNodeInfo.setNodeInformation(collection);
-        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
             System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
new file mode 100644
index 0000000..a744905
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.remote.util.StandardDataPacket;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestSiteToSiteClient {
+
+    @Test
+    @Ignore("For local testing only; not really a unit test but a manual test")
+    public void testReceive() throws IOException {
+        System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
+        
+        final SiteToSiteClient client = new SiteToSiteClient.Builder()
+            .url("http://localhost:8080/nifi")
+            .portName("out")
+            .requestBatchCount(1)
+            .build();
+        
+        try {
+            final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
+            Assert.assertNotNull(transaction);
+            
+            final DataPacket packet = transaction.receive();
+            Assert.assertNotNull(packet);
+            
+            final InputStream in = packet.getData();
+            final long size = packet.getSize();
+            final byte[] buff = new byte[(int) size];
+            
+            StreamUtils.fillBuffer(in, buff);
+            System.out.println(buff.length);
+            
+            Assert.assertNull(transaction.receive());
+            
+            transaction.confirm();
+            transaction.complete(false);
+        } finally {
+            client.close();
+        }
+    }
+    
+    
+    @Test
+    @Ignore("For local testing only; not really a unit test but a manual test")
+    public void testSend() throws IOException {
+        System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
+        
+        final SiteToSiteClient client = new SiteToSiteClient.Builder()
+            .url("http://localhost:8080/nifi")
+            .portName("in")
+            .build();
+        
+        try {
+            final Transaction transaction = client.createTransaction(TransferDirection.SEND);
+            Assert.assertNotNull(transaction);
+            
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("site-to-site", "yes, please!");
+            final byte[] bytes = "Hello".getBytes();
+            final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+            final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
+            transaction.send(packet);
+
+            transaction.confirm();
+            transaction.complete(false);
+        } finally {
+            client.close();
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
index 32a3f26..f68c874 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java
@@ -43,6 +43,16 @@ public class SocketChannelInputStream extends InputStream {
     public void setTimeout(final int timeoutMillis) {
         this.timeoutMillis = timeoutMillis;
     }
+    
+    public void consume() throws IOException {
+        final byte[] b = new byte[4096];
+        final ByteBuffer buffer = ByteBuffer.wrap(b);
+        int bytesRead;
+        do {
+            bytesRead = channel.read(buffer);
+            buffer.flip();
+        } while ( bytesRead > 0 );
+    }
 
     @Override
     public int read() throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
index 5810488..7c74b20 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
@@ -258,6 +258,16 @@ public class SSLSocketChannel implements Closeable {
         }
     }
 
+    public void consume() throws IOException {
+        final byte[] b = new byte[4096];
+        final ByteBuffer buffer = ByteBuffer.wrap(b);
+        int readCount;
+        do {
+            readCount = channel.read(buffer);
+            buffer.flip();
+        } while (readCount > 0);
+    }
+    
     private int readData(final ByteBuffer dest) throws IOException {
         final long startTime = System.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
index 154bd08..6fb79d4 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java
@@ -27,6 +27,10 @@ public class SSLSocketChannelInputStream extends InputStream {
         this.channel = channel;
     }
 
+    public void consume() throws IOException {
+        channel.consume();
+    }
+    
     @Override
     public int read() throws IOException {
         return channel.read();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
index ac41cba..c842195 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
@@ -24,9 +24,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.nifi.connectable.Position;
 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.client.socket.EndpointConnectionStatePool;
 
 public interface RemoteProcessGroup {
 
@@ -81,8 +79,6 @@ public interface RemoteProcessGroup {
 
     String getYieldDuration();
     
-    EndpointConnectionStatePool getConnectionPool();
-
     /**
      * Sets the timeout using the TimePeriod format (e.g., "30 secs", "1 min")
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 7cb2874..54f0807 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -48,6 +48,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import javax.net.ssl.SSLContext;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
@@ -128,14 +129,12 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
 import org.apache.nifi.groups.StandardProcessGroup;
-import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepository;
 import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.logging.ProcessorLogObserver;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarClassLoader;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.nar.NarThreadContextClassLoader;
 import org.apache.nifi.processor.Processor;
@@ -165,6 +164,7 @@ import org.apache.nifi.reporting.EventAccess;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.ReflectionUtils;
@@ -184,7 +184,6 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-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/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index db0aeb7..79ef7a8 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -56,7 +56,6 @@ 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.remote.client.socket.EndpointConnectionStatePool;
 import org.apache.nifi.remote.util.RemoteNiFiUtils;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.Severity;
@@ -130,7 +129,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
 
     private volatile String authorizationIssue;
 
-    private final EndpointConnectionStatePool endpointConnectionPool;
     private final ScheduledExecutorService backgroundThreadExecutor;
 
     public StandardRemoteProcessGroup(final String id, final String targetUri, final ProcessGroup processGroup,
@@ -172,13 +170,9 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
             }
         };
 
-        endpointConnectionPool = new EndpointConnectionStatePool(getTargetUri().toString(), getCommunicationsTimeout(TimeUnit.MILLISECONDS), 
-        		sslContext, eventReporter, getPeerPersistenceFile());
-        
         final Runnable checkAuthorizations = new InitializationTask();
-
         backgroundThreadExecutor = new FlowEngine(1, "Remote Process Group " + id + ": " + targetUri);
-        backgroundThreadExecutor.scheduleWithFixedDelay(checkAuthorizations, 0L, 30L, TimeUnit.SECONDS);
+        backgroundThreadExecutor.scheduleWithFixedDelay(checkAuthorizations, 5L, 30L, TimeUnit.SECONDS);
     }
 
     @Override
@@ -200,7 +194,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     @Override
     public void shutdown() {
         backgroundThreadExecutor.shutdown();
-        endpointConnectionPool.shutdown();
     }
     
     @Override
@@ -1222,11 +1215,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
     
     @Override
-    public EndpointConnectionStatePool getConnectionPool() {
-        return endpointConnectionPool;
-    }
-
-    @Override
     public void verifyCanDelete() {
         verifyCanDelete(false);
     }


[03/29] incubator-nifi git commit: NIFI-282: Bug fixes; documentation improvements; removed code marked as 'FOR TESTING'

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java
new file mode 100644
index 0000000..2e93599
--- /dev/null
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.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.stream.io;
+
+import java.io.EOFException;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * An InputStream that will throw EOFException if the underlying InputStream runs out of data before reaching the
+ * configured minimum amount of data
+ */
+public class MinimumLengthInputStream extends FilterInputStream {
+
+	private final long minLength;
+	private long consumedCount = 0L;
+	
+	public MinimumLengthInputStream(final InputStream in, final long minLength) {
+		super(in);
+		this.minLength = minLength;
+	}
+
+	
+	@Override
+	public int read() throws IOException {
+		final int b = super.read();
+		if ( b < 0 && consumedCount < minLength ) {
+			throw new EOFException();
+		}
+		
+		if ( b >= 0 ) {
+			consumedCount++;
+		}
+		
+		return b;
+	}
+	
+	@Override
+	public int read(byte[] b) throws IOException {
+		return read(b, 0, b.length);
+	}
+	
+	public int read(byte[] b, int off, int len) throws IOException {
+		final int num = super.read(b, off, len);
+		
+		if ( num < 0 && consumedCount < minLength ) {
+			throw new EOFException();
+		}
+		
+		if ( num >= 0 ) {
+			consumedCount += num;
+		}
+
+		return num;
+	}
+	
+	@Override
+	public long skip(final long n) throws IOException {
+		long skipped = super.skip(n);
+		if ( skipped < 1 ) {
+			final int b = super.read();
+			if ( b >= 0 ) {
+				skipped = 1;
+			}
+		}
+		
+		if ( skipped < 0 && consumedCount < minLength ) {
+			throw new EOFException();
+		}
+		
+		if ( skipped >= 0 ) {
+			consumedCount += skipped;
+		}
+		
+		return skipped;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/pom.xml b/nifi/nifi-commons/pom.xml
index ec0bb62..768dfd9 100644
--- a/nifi/nifi-commons/pom.xml
+++ b/nifi/nifi-commons/pom.xml
@@ -35,5 +35,6 @@
         <module>nifi-web-utils</module>
         <module>nifi-processor-utilities</module>
         <module>nifi-write-ahead-log</module>
+		<module>nifi-site-to-site-client</module>
     </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml
index 4a75a34..3d9f3ac 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml
@@ -74,6 +74,10 @@
             <artifactId>nifi-site-to-site</artifactId>
         </dependency>
         <dependency>
+        	<groupId>org.apache.nifi</groupId>
+        	<artifactId>nifi-site-to-site-client</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-compress</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
index 0fcac8c..3a1dfb2 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java
@@ -16,15 +16,17 @@
  */
 package org.apache.nifi.cluster.manager;
 
-import org.apache.nifi.cluster.manager.exception.PrimaryRoleAssignmentException;
-import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
-import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
-import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.nifi.cluster.event.Event;
 import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
+import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException;
 import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException;
+import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException;
 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.manager.exception.PrimaryRoleAssignmentException;
+import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.node.Node;
 import org.apache.nifi.cluster.node.Node.Status;
 import org.apache.nifi.cluster.protocol.ConnectionRequest;
@@ -33,11 +35,9 @@ 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;
-import java.util.Set;
-
 /**
  * Defines the interface for a ClusterManager. The cluster manager is a
  * threadsafe centralized manager for a cluster. Members of a cluster are nodes.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
index 656759a..4d5455f 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
@@ -63,11 +63,10 @@ import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 import javax.xml.validation.Validator;
 
+import org.apache.commons.lang3.StringUtils;
 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;
@@ -117,6 +116,7 @@ import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
 import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage;
 import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
 import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType;
+import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
 import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ControllerService;
@@ -155,6 +155,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 +193,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;
 import org.w3c.dom.DOMException;
@@ -203,8 +204,6 @@ import org.xml.sax.SAXParseException;
 
 import com.sun.jersey.api.client.ClientResponse;
 
-import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage;
-
 /**
  * Provides a cluster manager implementation. The manager federates incoming
  * HTTP client requests to the nodes' external API using the HTTP protocol. The

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
index d088c5c..4b4536e 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
@@ -44,6 +44,10 @@
             <artifactId>nifi-api</artifactId>
         </dependency>
         <dependency>
+        	<groupId>org.apache.nifi</groupId>
+        	<artifactId>nifi-site-to-site-client</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-lang3</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java
deleted file mode 100644
index 0092f7a..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java
deleted file mode 100644
index 5751c32..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java
deleted file mode 100644
index 987ff65..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java
deleted file mode 100644
index 848eb7e..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java
deleted file mode 100644
index 630631f..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
index e0cca64..ac41cba 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.groups;
 
-import java.io.IOException;
 import java.net.URI;
 import java.util.Date;
 import java.util.Set;
@@ -27,7 +26,7 @@ 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;
+import org.apache.nifi.remote.client.socket.EndpointConnectionStatePool;
 
 public interface RemoteProcessGroup {
 
@@ -81,6 +80,8 @@ public interface RemoteProcessGroup {
     void setYieldDuration(final String yieldDuration);
 
     String getYieldDuration();
+    
+    EndpointConnectionStatePool getConnectionPool();
 
     /**
      * Sets the timeout using the TimePeriod format (e.g., "30 secs", "1 min")
@@ -107,15 +108,6 @@ public interface RemoteProcessGroup {
     String getCommunicationsTimeout();
 
     /**
-     * @return the port that the remote instance is listening on for
-     * site-to-site communication, or <code>null</code> if the remote instance
-     * is not configured to allow site-to-site communications.
-     *
-     * @throws IOException if unable to communicate with the remote instance
-     */
-    Integer getListeningPort() throws IOException;
-
-    /**
      * Indicates whether or not the RemoteProcessGroup is currently scheduled to
      * transmit data
      *
@@ -211,11 +203,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
@@ -232,24 +219,4 @@ public interface RemoteProcessGroup {
     void verifyCanStopTransmitting();
 
     void verifyCanUpdate();
-
-    /**
-     * Returns a set of PeerStatus objects that describe the different peers
-     * that we can communicate with for this RemoteProcessGroup.
-     *
-     * If the destination is a cluster, this set will contain PeerStatuses for
-     * each of the nodes in the cluster.
-     *
-     * If the destination is a standalone instance, this set will contain just a
-     * PeerStatus for the destination.
-     *
-     * Once the PeerStatuses have been obtained, they may be cached by this
-     * RemoteProcessGroup for some amount of time.
-     *
-     * If unable to obtain the PeerStatuses or no peer status has yet been
-     * obtained, will return null.
-     *
-     * @return
-     */
-    Set<PeerStatus> getPeerStatuses();
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/Peer.java
deleted file mode 100644
index 2422fe1..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/PeerStatus.java
deleted file mode 100644
index d1cb076..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
index d4ad374..f08277c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/TransferDirection.java
deleted file mode 100644
index 56432d5..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
deleted file mode 100644
index bfccd98..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/cluster/NodeInformant.java
new file mode 100644
index 0000000..e46ff5c
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
deleted file mode 100644
index b4206b3..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
deleted file mode 100644
index b61fc65..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
deleted file mode 100644
index af0f467..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
deleted file mode 100644
index 0f50b98..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
deleted file mode 100644
index e6a0fe7..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
deleted file mode 100644
index 32274eb..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
deleted file mode 100644
index d2e2946..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
deleted file mode 100644
index 95cab29..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
deleted file mode 100644
index d009cec..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
deleted file mode 100644
index 41334fe..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
index 0d18f2e..0118534 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index df5c845..1ff5fbe 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -64,6 +64,10 @@
             <artifactId>nifi-client-dto</artifactId>
         </dependency>
         <dependency>
+        	<groupId>org.apache.nifi</groupId>
+        	<artifactId>nifi-site-to-site-client</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.quartz-scheduler</groupId>
             <artifactId>quartz</artifactId>
         </dependency>


[29/29] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/705ee852
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/705ee852
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/705ee852

Branch: refs/heads/develop
Commit: 705ee852bb79dbfa5662ffa9733cbcc7b12410be
Parents: a00aade 42f6919
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 23 14:56:41 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 23 14:56:41 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/web/api/RemoteProcessGroupResource.java  | 8 ++++++++
 .../partials/canvas/new-remote-process-group-dialog.jsp      | 2 +-
 2 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[06/29] incubator-nifi git commit: NIFI-282: Bug fixes; documentation improvements; removed code marked as 'FOR TESTING'

Posted by ma...@apache.org.
NIFI-282: Bug fixes; documentation improvements; removed code marked as 'FOR TESTING'


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/05b64593
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/05b64593
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/05b64593

Branch: refs/heads/develop
Commit: 05b64593b6958aa807066e0df1571becf589dc17
Parents: ed53b46
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Feb 6 08:19:54 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Feb 6 08:19:54 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/remote/RemoteDestination.java   |  47 ++
 .../nifi-site-to-site-client/pom.xml            |  43 +
 .../remote/AbstractCommunicationsSession.java   |  54 ++
 .../main/java/org/apache/nifi/remote/Peer.java  | 119 +++
 .../java/org/apache/nifi/remote/PeerStatus.java |  72 ++
 .../nifi/remote/RemoteResourceInitiator.java    |  64 ++
 .../org/apache/nifi/remote/Transaction.java     | 200 +++++
 .../apache/nifi/remote/TransferDirection.java   |  34 +
 .../nifi/remote/VersionedRemoteResource.java    |  24 +
 .../nifi/remote/client/SiteToSiteClient.java    | 443 ++++++++++
 .../remote/client/SiteToSiteClientConfig.java   | 114 +++
 .../client/socket/EndpointConnectionState.java  |  54 ++
 .../socket/EndpointConnectionStatePool.java     | 835 +++++++++++++++++++
 .../nifi/remote/client/socket/SocketClient.java | 184 ++++
 .../remote/cluster/AdaptedNodeInformation.java  |  66 ++
 .../remote/cluster/ClusterNodeInformation.java  |  67 ++
 .../nifi/remote/cluster/NodeInformation.java    |  98 +++
 .../remote/cluster/NodeInformationAdapter.java  |  41 +
 .../apache/nifi/remote/codec/FlowFileCodec.java |  71 ++
 .../remote/codec/StandardFlowFileCodec.java     | 129 +++
 .../remote/exception/HandshakeException.java    |  30 +
 .../exception/PortNotRunningException.java      |  26 +
 .../remote/exception/ProtocolException.java     |  36 +
 .../remote/exception/UnknownPortException.java  |  26 +
 .../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    |  86 ++
 .../remote/protocol/CommunicationsInput.java    |  27 +
 .../remote/protocol/CommunicationsOutput.java   |  27 +
 .../remote/protocol/CommunicationsSession.java  |  64 ++
 .../apache/nifi/remote/protocol/DataPacket.java |  29 +
 .../nifi/remote/protocol/RequestType.java       |  43 +
 .../protocol/socket/HandshakeProperty.java      |  61 ++
 .../nifi/remote/protocol/socket/Response.java   |  51 ++
 .../remote/protocol/socket/ResponseCode.java    | 153 ++++
 .../protocol/socket/SocketClientProtocol.java   | 437 ++++++++++
 .../socket/SocketClientTransaction.java         | 357 ++++++++
 .../nifi/remote/util/PeerStatusCache.java       |  43 +
 .../nifi/remote/util/RemoteNiFiUtils.java       | 216 +++++
 .../nifi/remote/util/StandardDataPacket.java    |  50 ++
 .../socket/TestEndpointConnectionStatePool.java |  95 +++
 .../nifi/stream/io/LimitingInputStream.java     | 111 +++
 .../stream/io/MinimumLengthInputStream.java     |  93 +++
 nifi/nifi-commons/pom.xml                       |   1 +
 .../nifi-framework/nifi-cluster/pom.xml         |   4 +
 .../nifi/cluster/manager/ClusterManager.java    |  18 +-
 .../cluster/manager/impl/WebClusterManager.java |   9 +-
 .../nifi-framework-core-api/pom.xml             |   4 +
 .../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  |  39 +-
 .../main/java/org/apache/nifi/remote/Peer.java  | 107 ---
 .../java/org/apache/nifi/remote/PeerStatus.java |  72 --
 .../org/apache/nifi/remote/RemoteGroupPort.java |  22 +-
 .../apache/nifi/remote/TransferDirection.java   |  23 -
 .../nifi/remote/VersionedRemoteResource.java    |  24 -
 .../nifi/remote/cluster/NodeInformant.java      |  22 +
 .../apache/nifi/remote/codec/FlowFileCodec.java |  79 --
 .../remote/exception/HandshakeException.java    |  30 -
 .../exception/PortNotRunningException.java      |  26 -
 .../remote/exception/ProtocolException.java     |  34 -
 .../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-framework/nifi-framework-core/pom.xml  |   4 +
 .../nifi/remote/StandardRemoteProcessGroup.java | 316 +------
 .../nifi-framework/nifi-site-to-site/.gitignore |   1 +
 .../util/RemoteProcessGroupUtils.class          | Bin 0 -> 9526 bytes
 .../remote/AbstractCommunicationsSession.class  | Bin 0 -> 2308 bytes
 .../nifi/remote/RemoteResourceFactory.class     | Bin 0 -> 8707 bytes
 .../nifi/remote/RemoteResourceManager.class     | Bin 0 -> 6898 bytes
 .../apache/nifi/remote/RemoteSiteListener.class | Bin 0 -> 841 bytes
 .../nifi/remote/SocketRemoteSiteListener.class  | Bin 0 -> 8448 bytes
 ...emoteGroupPort$EndpointConnectionState.class | Bin 0 -> 5427 bytes
 .../nifi/remote/StandardRemoteGroupPort.class   | Bin 0 -> 10677 bytes
 .../StandardRootGroupPort$FlowFileRequest.class | Bin 0 -> 5836 bytes
 ...StandardRootGroupPort$ProcessingResult.class | Bin 0 -> 5032 bytes
 ...upPort$StandardPortAuthorizationResult.class | Bin 0 -> 5159 bytes
 .../nifi/remote/StandardRootGroupPort.class     | Bin 0 -> 9700 bytes
 .../remote/codec/StandardFlowFileCodec.class    | Bin 0 -> 8538 bytes
 .../exception/UnsupportedCodecException.class   | Bin 0 -> 1057 bytes
 .../SocketChannelCommunicationsSession.class    | Bin 0 -> 3735 bytes
 .../remote/io/socket/SocketChannelInput.class   | Bin 0 -> 4008 bytes
 .../remote/io/socket/SocketChannelOutput.class  | Bin 0 -> 3741 bytes
 .../SSLSocketChannelCommunicationsSession.class | Bin 0 -> 4611 bytes
 .../io/socket/ssl/SSLSocketChannelInput.class   | Bin 0 -> 3127 bytes
 .../io/socket/ssl/SSLSocketChannelOutput.class  | Bin 0 -> 2587 bytes
 .../socket/ClusterManagerServerProtocol.class   | Bin 0 -> 10540 bytes
 .../protocol/socket/HandshakeProperty.class     | Bin 0 -> 917 bytes
 .../nifi/remote/protocol/socket/Response.class  | Bin 0 -> 2674 bytes
 .../remote/protocol/socket/ResponseCode.class   | Bin 0 -> 6889 bytes
 .../protocol/socket/SocketClientProtocol.class  | Bin 0 -> 8965 bytes
 .../socket/SocketFlowFileServerProtocol.class   | Bin 0 -> 8806 bytes
 .../remote/TestStandardRemoteGroupPort.class    | Bin 0 -> 5974 bytes
 .../nifi-framework/nifi-site-to-site/pom.xml    |   4 +
 .../remote/AbstractCommunicationsSession.java   |  54 --
 .../nifi/remote/RemoteResourceFactory.java      |  50 +-
 .../nifi/remote/SocketRemoteSiteListener.java   |   7 +-
 .../nifi/remote/StandardRemoteGroupPort.java    | 485 +----------
 .../remote/codec/StandardFlowFileCodec.java     | 169 ----
 .../SocketChannelCommunicationsSession.java     |  90 --
 .../remote/io/socket/SocketChannelInput.java    |  66 --
 .../remote/io/socket/SocketChannelOutput.java   |  58 --
 .../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    | 193 +++--
 .../remote/TestStandardRemoteGroupPort.java     |  97 ---
 nifi/pom.xml                                    |   5 +
 122 files changed, 5501 insertions(+), 3135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/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..f718581
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+import java.util.concurrent.TimeUnit;
+
+
+/**
+ * A model object for referring to a remote destination (i.e., a Port) for site-to-site communications
+ */
+public interface RemoteDestination {
+    /**
+     * Returns the identifier of the remote destination
+     * 
+     * @return
+     */
+	String getIdentifier();
+	
+	/**
+	 * Returns the amount of time that system should pause sending to a particular node if unable to 
+	 * send data to or receive data from this endpoint
+	 * @param timeUnit
+	 * @return
+	 */
+	long getYieldPeriod(TimeUnit timeUnit);
+	
+	/**
+	 * Returns whether or not compression should be used when transferring data to or receiving
+	 * data from the remote endpoint
+	 * @return
+	 */
+	boolean isUseCompression();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml
new file mode 100644
index 0000000..3fc00a2
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml
@@ -0,0 +1,43 @@
+<?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</artifactId>
+    <version>0.0.2-incubating-SNAPSHOT</version>
+  </parent>
+  
+  <artifactId>nifi-site-to-site-client</artifactId>
+  
+  <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>com.sun.jersey</groupId>
+		<artifactId>jersey-client</artifactId>
+	</dependency>
+	<dependency>
+		<groupId>org.apache.nifi</groupId>
+		<artifactId>nifi-client-dto</artifactId>
+		<version>0.0.2-incubating-SNAPSHOT</version>
+	</dependency>
+  	<dependency>
+  		<groupId>org.apache.nifi</groupId>
+  		<artifactId>nifi-web-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
new file mode 100644
index 0000000..4babb92
--- /dev/null
+++ b/nifi/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
new file mode 100644
index 0000000..29af777
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+import org.apache.nifi.stream.io.NullOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+
+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;
+
+        // Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer
+        try {
+            StreamUtils.copy(commsSession.getInput().getInputStream(), new NullOutputStream());
+        } finally {
+            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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
new file mode 100644
index 0000000..d1cb076
--- /dev/null
+++ b/nifi/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
new file mode 100644
index 0000000..8eb5d8d
--- /dev/null
+++ b/nifi/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
new file mode 100644
index 0000000..cc16625
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.nifi.remote.protocol.DataPacket;
+
+
+/**
+ * <p>
+ * Provides a transaction for performing site-to-site data transfers.
+ * </p>
+ * 
+ * <p>
+ * A Transaction is created by calling the 
+ * {@link org.apache.nifi.remote.client.SiteToSiteClient#createTransaction(TransferDirection) createTransaction(TransferDirection)} 
+ * method of a {@link org.apache.nifi.remote.client.SiteToSiteClient SiteToSiteClient}. The resulting Transaction
+ * can be used to either send or receive data but not both. A new Transaction must be created in order perform the
+ * other operation.
+ * </p>
+ * 
+ * <p>
+ * The general flow of execute of a Transaction is as follows:
+ * <ol>
+ *      <li>Create the transaction as described above.</li>
+ *      <li>Send data via the {@link #send(DataPacket)} method or receive data via the {@link #receive()} method. This method
+ *          will be called 1 or more times. In the case of receive, this method should be called until the method returns {@code null},
+ *          signifying that the remote instance is finished sending data. <b>Note:</b> <code>receive()</code> should not be
+ *          called a second time without first fully consuming the stream from the previous Packet that was received.</li>
+ *      <li>Confirm the transaction via the {@link #confirm()} method.</li>
+ *      <li>Either complete the transaction via the {@link #complete(boolean)} method or cancel the transaction
+ *          via the {@link #cancel()} method.</li>
+ * </ol>
+ * </p>
+ * 
+ * <p>
+ * It is important that the Transaction be terminated in order to free the resources held
+ * by the Transaction. If a Transaction is not terminated, its resources will not be freed and
+ * if the Transaction holds connections from a connection pool, the connections in that pool
+ * will eventually become exhausted. A Transaction is terminated by calling one of the following
+ * methods:
+ *  <ul>
+ *      <li>{@link #complete(boolean)}</li>
+ *      <li>{@link #cancel()}</li>
+ *      <li>{@link #error()}</li>
+ *  </ul>
+ * </p>
+ * 
+ * <p>
+ * If at any point an IOException is thrown from one of the methods of the Transaction, that Transaction
+ * is automatically closed via a call to {@link #error()}.
+ * </p>
+ * 
+ * <p>
+ * The Transaction class should not be assumed to be thread-safe.
+ * </p>
+ */
+public interface Transaction {
+
+    /**
+     * Sends information to the remote NiFi instance.
+     * 
+     * @param dataPacket the data packet to send
+     * @throws IOException
+     */
+    void send(DataPacket dataPacket) throws IOException;
+    
+    /**
+     * Retrieves information from the remote NiFi instance, if any is available. If no data is available, will return
+     * {@code null}. It is important to consume all data from the remote NiFi instance before attempting to 
+     * call {@link #confirm()}. This is because the sender is always responsible for determining when the Transaction
+     * has finished. This is done in order to prevent the need for a round-trip network request to receive data for
+     * each data packet.
+     * 
+     * @return the DataPacket received, or {@code null} if there is no more data to receive. 
+     * @throws IOException
+     */
+    DataPacket receive() throws IOException;
+
+    /**
+     * <p>
+     * Confirms the data that was sent or received by comparing CRC32's of the data sent and the data received.
+     * </p>
+     * 
+     * <p>
+     * Even if the protocol being used to send the data is reliable and guarantees ordering of packets (such as TCP),
+     * it is still required that we confirm the transaction before completing the transaction. This is done as
+     * "safety net" or a defensive programming technique. Mistakes happen, and this mechanism helps to ensure that if
+     * a bug exists somewhere along the line that we do not end up sending or receiving corrupt data. If the
+     * CRC32 of the sender and the CRC32 of the receiver do not match, an IOException will be thrown and both the
+     * sender and receiver will cancel the transaction automatically.
+     * </p>
+     * 
+     * <p>
+     * If the {@link TransferDirection} of this Transaction is RECEIVE, this method will throw an Exception unless
+     * all data from the remote instance has been consumed (i.e., a call to {@link #receive()} returns {@code null}).
+     * </p>
+     * 
+     * <p>
+     * If the {@link TransferDirection} of this Transaction is SEND, calling this method dictates that no more data will be
+     * sent in this transaction. I.e., there will be no more calls to {@link #send(DataPacket)}.
+     * </p>
+     * 
+     * @throws IOException
+     */
+	void confirm() throws IOException;
+	
+	/**
+	 * <p>
+	 * Completes the transaction and indicates to both the sender and receiver that the data transfer was
+	 * successful. If receiving data, this method can also optionally request that the sender back off sending
+	 * data for a short period of time. This is used, for instance, to apply backpressure or to notify the sender
+	 * that the receiver is not ready to receive data and made not service another request in the short term.
+	 * </p>
+	 * 
+	 * @param requestBackoff if <code>true</code> and the TransferDirection is RECEIVE, indicates to sender that it
+	 * should back off sending data for a short period of time. If <code>false</code> or if the TransferDirection of
+	 * this Transaction is SEND, then this argument is ignored.
+	 * 
+	 * @throws IOException
+	 */
+	void complete(boolean requestBackoff) throws IOException;
+	
+	/**
+	 * <p>
+	 * Cancels this transaction, indicating to the sender that the data has not been successfully received so that
+	 * the sender can retry or handle however is appropriate.
+	 * </p>
+	 * 
+	 * @param explanation an explanation to tell the other party why the transaction was canceled.
+	 * @throws IOException
+	 */
+	void cancel(final String explanation) throws IOException;
+	
+	
+	/**
+	 * <p>
+	 * Sets the TransactionState of the Transaction to {@link TransactionState#ERROR}, and closes
+	 * the Transaction. The underlying connection should not be returned to a connection pool in this case.
+	 * </p>
+	 */
+	void error();
+	
+	
+	/**
+	 * Returns the current state of the Transaction.
+	 * @return
+	 * @throws IOException
+	 */
+	TransactionState getState() throws IOException;
+	
+	
+	public enum TransactionState {
+	    /**
+	     * Transaction has been started but no data has been sent or received.
+	     */
+		TRANSACTION_STARTED,
+		
+		/**
+		 * Transaction has been started and data has been sent or received.
+		 */
+		DATA_EXCHANGED,
+		
+		/**
+		 * Data that has been transferred has been confirmed via its CRC. Transaction is
+		 * ready to be completed.
+		 */
+		TRANSACTION_CONFIRMED,
+		
+		/**
+		 * Transaction has been successfully completed.
+		 */
+		TRANSACTION_COMPLETED,
+		
+		/**
+		 * The Transaction has been canceled.
+		 */
+		TRANSACTION_CANCELED,
+		
+		/**
+		 * The Transaction ended in an error.
+		 */
+		ERROR;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.java
new file mode 100644
index 0000000..45029a4
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransferDirection.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;
+
+
+/**
+ * An enumeration for specifying the direction in which data should be transferred between a client
+ * and a remote NiFi instance.
+ */
+public enum TransferDirection {
+	/**
+	 * The client is to send data to the remote instance.
+	 */
+    SEND,
+    
+    /**
+     * The client is to receive data from the remote instance.
+     */
+    RECEIVE;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/VersionedRemoteResource.java
new file mode 100644
index 0000000..bfccd98
--- /dev/null
+++ b/nifi/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
new file mode 100644
index 0000000..fa94b81
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.client.socket.SocketClient;
+import org.apache.nifi.remote.protocol.DataPacket;
+
+/**
+ * <p>
+ * The SiteToSiteClient provides a mechanism for sending data to a remote instance of NiFi
+ * (or NiFi cluster) and retrieving data from a remote instance of NiFi (or NiFi cluster).
+ * </p>
+ * 
+ * <p>
+ * When configuring the client via the {@link SiteToSiteClient.Builder}, the Builder must
+ * be provided the URL of the remote NiFi instance. If the URL points to a standalone instance
+ * of NiFi, all interaction will take place with that instance of NiFi. However, if the URL 
+ * points to the NiFi Cluster Manager of a cluster, the client will automatically handle load
+ * balancing the transactions across the different nodes in the cluster.
+ * </p>
+ * 
+ * <p>
+ * The SiteToSiteClient provides a {@link Transaction} through which all interaction with the
+ * remote instance takes place. After data has been exchanged or it is determined that no data
+ * is available, the Transaction can then be canceled (via the {@link Transaction#cancel(String)}
+ * method) or can be completed (via the {@link Transaction#complete(boolean)} method).
+ * </p>
+ * 
+ * <p>
+ * An instance of SiteToSiteClient can be obtained by constructing a new instance of the 
+ * {@link SiteToSiteClient.Builder} class, calling the appropriate methods to configured the
+ * client as desired, and then calling the {@link SiteToSiteClient.Builder#build() build()} method.
+ * </p>
+ *
+ * <p>
+ * The SiteToSiteClient itself is immutable once constructed and is thread-safe. Many threads can
+ * share access to the same client. However, the {@link Transaction} that is created by the client
+ * is not thread safe and should not be shared among threads.
+ * </p>
+ */
+public interface SiteToSiteClient extends Closeable {
+
+	/**
+	 * Creates a new Transaction that can be used to either send data to a remote NiFi instance
+	 * or receive data from a remote NiFi instance, depending on the value passed for the {@code direction} argument.
+	 * 
+	 * 
+	 * @param direction specifies which direction the data should be transferred. A value of {@link TransferDirection#SEND}
+	 * indicates that this Transaction will send data to the remote instance; a value of {@link TransferDirection#RECEIVE} indicates
+	 * that this Transaction will be used to receive data from the remote instance.
+	 * 
+	 * @return
+	 * @throws IOException
+	 */
+	Transaction createTransaction(TransferDirection direction) throws IOException;
+	
+	/**
+	 * <p>
+	 * Returns {@code true} if site-to-site communications with the remote instance are secure, 
+	 * {@code false} if site-to-site communications with the remote instance are not secure. Whether or not
+	 * communications are secure depends on the server, not the client.
+	 * </p>
+	 * 
+	 * <p>
+	 * In order to determine whether the server is configured for secure communications, the client may have
+	 * to query the server's RESTful interface. Doing so could result in an IOException.
+	 * </p>
+	 * 
+	 * @return
+	 * @throws IOException if unable to query the remote instance's RESTful interface or if the remote
+	 * instance is not configured to allow site-to-site communications
+	 */
+	boolean isSecure() throws IOException;
+	
+	/**
+	 * <p>
+	 * Returns the configuration object that was built by the Builder
+	 * </p>
+	 * @return
+	 */
+	SiteToSiteClientConfig getConfig();
+	
+	/**
+	 * <p>
+	 * The Builder is the mechanism by which all configuration is passed to the SiteToSiteClient.
+	 * Once constructed, the SiteToSiteClient cannot be reconfigured (i.e., it is immutable). If
+	 * a change in configuration should be desired, the client should be {@link Closeable#close() closed}
+	 * and a new client created. 
+	 * </p>
+	 */
+	public static class Builder {
+		private String url;
+		private long timeoutNanos = TimeUnit.SECONDS.toNanos(30);
+		private long penalizationNanos = TimeUnit.SECONDS.toNanos(3);
+		private SSLContext sslContext;
+		private EventReporter eventReporter;
+		private File peerPersistenceFile;
+		private boolean useCompression;
+		private String portName;
+		private String portIdentifier;
+		private int batchCount;
+		private long batchSize;
+		private long batchNanos;
+
+		/**
+		 * Specifies the URL of the remote NiFi instance. If this URL points to the Cluster Manager of
+		 * a NiFi cluster, data transfer to and from nodes will be automatically load balanced across
+		 * the different nodes.
+		 * 
+		 * @param url
+		 * @return
+		 */
+		public Builder url(final String url) {
+			this.url = url;
+			return this;
+		}
+		
+		/**
+		 * Specifies the communications timeouts to use when interacting with the remote instances. The
+		 * default value is 30 seconds.
+		 * 
+		 * @param timeout
+		 * @param unit
+		 * @return
+		 */
+		public Builder timeout(final long timeout, final TimeUnit unit) {
+			this.timeoutNanos = unit.toNanos(timeout);
+			return this;
+		}
+		
+		/**
+		 * If there is a problem communicating with a node (i.e., any node in the remote NiFi cluster
+		 * or the remote instance of NiFi if it is standalone), specifies how long the client should
+		 * wait before attempting to communicate with that node again. While a particular node is penalized,
+		 * all other nodes in the remote cluster (if any) will still be available for communication.
+		 * The default value is 3 seconds.
+		 * 
+		 * @param period
+		 * @param unit
+		 * @return
+		 */
+		public Builder nodePenalizationPeriod(final long period, final TimeUnit unit) {
+			this.penalizationNanos = unit.toNanos(period);
+			return this;
+		}
+		
+		/**
+		 * Specifies the SSL Context to use when communicating with the remote NiFi instance(s). If not
+		 * specified, communications will not be secure. The remote instance of NiFi always determines
+		 * whether or not Site-to-Site communications are secure (i.e., the client will always use
+		 * secure or non-secure communications, depending on what the server dictates).
+		 * 
+		 * @param sslContext
+		 * @return
+		 */
+		public Builder sslContext(final SSLContext sslContext) {
+			this.sslContext = sslContext;
+			return this;
+		}
+		
+		
+		/**
+		 * Provides an EventReporter that can be used by the client in order to report any events that
+		 * could be of interest when communicating with the remote instance. The EventReporter provided
+		 * must be threadsafe.
+		 * 
+		 * @param eventReporter
+		 * @return
+		 */
+		public Builder eventReporter(final EventReporter eventReporter) {
+			this.eventReporter = eventReporter;
+			return this;
+		}
+		
+		
+		/**
+		 * Specifies a file that the client can write to in order to persist the list of nodes in the
+		 * remote cluster and recover the list of nodes upon restart. This allows the client to function
+		 * if the remote Cluster Manager is unavailable, even after a restart of the client software.
+		 * If not specified, the list of nodes will not be persisted and a failure of the Cluster Manager
+		 * will result in not being able to communicate with the remote instance if a new client
+		 * is created. 
+		 * 
+		 * @param peerPersistenceFile
+		 * @return
+		 */
+		public Builder peerPersistenceFile(final File peerPersistenceFile) {
+			this.peerPersistenceFile = peerPersistenceFile;
+			return this;
+		}
+		
+		/**
+		 * Specifies whether or not data should be compressed before being transferred to or from the
+		 * remote instance.
+		 * 
+		 * @param compress
+		 * @return
+		 */
+		public Builder useCompression(final boolean compress) {
+			this.useCompression = compress;
+			return this;
+		}
+		
+		/**
+		 * Specifies the name of the port to communicate with. Either the port name or the port identifier
+		 * must be specified.
+		 * 
+		 * @param portName
+		 * @return
+		 */
+		public Builder portName(final String portName) {
+			this.portName = portName;
+			return this;
+		}
+		
+		/**
+		 * Specifies the unique identifier of the port to communicate with. If it is known, this is preferred over providing
+		 * the port name, as the port name may change.
+		 * 
+		 * @param portIdentifier
+		 * @return
+		 */
+		public Builder portIdentifier(final String portIdentifier) {
+			this.portIdentifier = portIdentifier;
+			return this;
+		}
+		
+		/**
+	     * When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
+	     * the client has the ability to request a particular batch size/duration. This method specifies
+	     * the preferred number of {@link DataPacket}s to include in a Transaction.
+	     * 
+	     * @return
+	     */
+		public Builder requestBatchCount(final int count) {
+		    this.batchCount = count;
+		    return this;
+		}
+
+		/**
+	     * When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
+	     * the client has the ability to request a particular batch size/duration. This method specifies
+	     * the preferred number of bytes to include in a Transaction.
+	     * 
+	     * @return
+	     */
+		public Builder requestBatchSize(final long bytes) {
+		    this.batchSize = bytes;
+		    return this;
+		}
+		
+        /**
+         * When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
+         * the client has the ability to request a particular batch size/duration. This method specifies
+         * the preferred amount of time that a Transaction should span.
+         * 
+         * @return
+         */
+		public Builder requestBatchDuration(final long value, final TimeUnit unit) {
+		    this.batchNanos = unit.toNanos(value);
+		    return this;
+		}
+		
+		
+		/**
+		 * Builds a new SiteToSiteClient that can be used to send and receive data with remote instances of NiFi
+		 * @return
+		 */
+		public SiteToSiteClient build() {
+			if ( url == null ) {
+				throw new IllegalStateException("Must specify URL to build Site-to-Site client");
+			}
+			
+			if ( portName == null && portIdentifier == null ) {
+				throw new IllegalStateException("Must specify either Port Name or Port Identifier to builder Site-to-Site client");
+			}
+			
+			final SiteToSiteClientConfig config = new SiteToSiteClientConfig() {
+				
+				@Override
+				public boolean isUseCompression() {
+					return Builder.this.isUseCompression();
+				}
+				
+				@Override
+				public String getUrl() {
+					return Builder.this.getUrl();
+				}
+				
+				@Override
+				public long getTimeout(final TimeUnit timeUnit) {
+					return Builder.this.getTimeout(timeUnit);
+				}
+				
+				@Override
+				public SSLContext getSslContext() {
+					return Builder.this.getSslContext();
+				}
+				
+				@Override
+				public String getPortName() {
+					return Builder.this.getPortName();
+				}
+				
+				@Override
+				public String getPortIdentifier() {
+					return Builder.this.getPortIdentifier();
+				}
+				
+				@Override
+				public long getPenalizationPeriod(final TimeUnit timeUnit) {
+					return Builder.this.getPenalizationPeriod(timeUnit);
+				}
+				
+				@Override
+				public File getPeerPersistenceFile() {
+					return Builder.this.getPeerPersistenceFile();
+				}
+				
+				@Override
+				public EventReporter getEventReporter() {
+					return Builder.this.getEventReporter();
+				}
+
+		        @Override
+		        public long getPreferredBatchDuration(final TimeUnit timeUnit) {
+		            return timeUnit.convert(Builder.this.batchNanos, TimeUnit.NANOSECONDS);
+		        }
+		        
+		        @Override
+		        public long getPreferredBatchSize() {
+		            return Builder.this.batchSize;
+		        }
+		        
+		        @Override
+		        public int getPreferredBatchCount() {
+		            return Builder.this.batchCount;
+		        }
+			};
+			
+			return new SocketClient(config);
+		}
+
+		/**
+		 * Returns the configured URL for the remote NiFi instance
+		 * @return
+		 */
+		public String getUrl() {
+			return url;
+		}
+
+		/**
+		 * Returns the communications timeout in nanoseconds
+		 * @return
+		 */
+		public long getTimeout(final TimeUnit timeUnit) {
+			return timeUnit.convert(timeoutNanos, TimeUnit.NANOSECONDS);
+		}
+
+		/**
+		 * Returns the amount of time that a particular node will be ignored after a
+		 * communications error with that node occurs
+		 * @param timeUnit
+		 * @return
+		 */
+		public long getPenalizationPeriod(TimeUnit timeUnit) {
+			return timeUnit.convert(penalizationNanos, TimeUnit.NANOSECONDS);
+		}
+
+		/**
+		 * Returns the SSL Context that is configured for this builder
+		 * @return
+		 */
+		public SSLContext getSslContext() {
+			return sslContext;
+		}
+
+		/**
+		 * Returns the EventReporter that is to be used by clients to report events
+		 * @return
+		 */
+		public EventReporter getEventReporter() {
+			return eventReporter;
+		}
+
+		/**
+		 * Returns the file that is to be used for persisting the nodes of a remote cluster, if any.
+		 * @return
+		 */
+		public File getPeerPersistenceFile() {
+			return peerPersistenceFile;
+		}
+
+		/**
+		 * Returns a boolean indicating whether or not compression will be used to transfer data
+		 * to and from the remote instance
+		 * @return
+		 */
+		public boolean isUseCompression() {
+			return useCompression;
+		}
+
+		/**
+		 * Returns the name of the port that the client is to communicate with.
+		 * @return
+		 */
+		public String getPortName() {
+			return portName;
+		}
+
+		/**
+		 * Returns the identifier of the port that the client is to communicate with.
+		 * @return
+		 */
+		public String getPortIdentifier() {
+			return portIdentifier;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
new file mode 100644
index 0000000..37c48f8
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.client;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.remote.protocol.DataPacket;
+
+public interface SiteToSiteClientConfig {
+
+	/**
+	 * Returns the configured URL for the remote NiFi instance
+	 * @return
+	 */
+	String getUrl();
+
+	/**
+	 * Returns the communications timeout in nanoseconds
+	 * @return
+	 */
+	long getTimeout(final TimeUnit timeUnit);
+
+	/**
+	 * Returns the amount of time that a particular node will be ignored after a
+	 * communications error with that node occurs
+	 * @param timeUnit
+	 * @return
+	 */
+	long getPenalizationPeriod(TimeUnit timeUnit);
+
+	/**
+	 * Returns the SSL Context that is configured for this builder
+	 * @return
+	 */
+	SSLContext getSslContext();
+	
+	/**
+	 * Returns the EventReporter that is to be used by clients to report events
+	 * @return
+	 */
+	EventReporter getEventReporter();
+
+	/**
+	 * Returns the file that is to be used for persisting the nodes of a remote cluster, if any.
+	 * @return
+	 */
+	File getPeerPersistenceFile();
+
+	/**
+	 * Returns a boolean indicating whether or not compression will be used to transfer data
+	 * to and from the remote instance
+	 * @return
+	 */
+	boolean isUseCompression();
+
+	/**
+	 * Returns the name of the port that the client is to communicate with.
+	 * @return
+	 */
+	String getPortName();
+
+	/**
+	 * Returns the identifier of the port that the client is to communicate with.
+	 * @return
+	 */
+	String getPortIdentifier();
+	
+	/**
+	 * When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
+	 * the client has the ability to request a particular batch size/duration. This returns the maximum
+	 * amount of time that we will request a NiFi instance to send data to us in a Transaction.
+	 * 
+	 * @param timeUnit
+	 * @return
+	 */
+	long getPreferredBatchDuration(TimeUnit timeUnit);
+	
+    /**
+     * When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
+     * the client has the ability to request a particular batch size/duration. This returns the maximum
+     * number of bytes that we will request a NiFi instance to send data to us in a Transaction.
+     * 
+     * @return
+     */
+	long getPreferredBatchSize();
+	
+	
+	/**
+     * When pulling data from a NiFi instance, the sender chooses how large a Transaction is. However,
+     * the client has the ability to request a particular batch size/duration. This returns the maximum
+     * number of {@link DataPacket}s that we will request a NiFi instance to send data to us in a Transaction.
+     * 
+     * @return
+     */
+	int getPreferredBatchCount();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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;
+    }
+}


[24/29] incubator-nifi git commit: NIFI-362: Ensure that we synchronize on ScheduleState before modifying it

Posted by ma...@apache.org.
NIFI-362: Ensure that we synchronize on ScheduleState before modifying it


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f1e74cc0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f1e74cc0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f1e74cc0

Branch: refs/heads/develop
Commit: f1e74cc04199354a6688e15de3c9947274992caa
Parents: 4cc106a
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Feb 22 10:59:14 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Feb 22 10:59:14 2015 -0500

----------------------------------------------------------------------
 .../scheduling/TimerDrivenSchedulingAgent.java    | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f1e74cc0/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
index efa8acd..a620202 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
@@ -132,8 +132,13 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
                             final long yieldNanos = TimeUnit.MILLISECONDS.toNanos(yieldMillis);
                             final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, yieldNanos, 
                                     connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
-                            scheduleState.replaceFuture(scheduledFuture, newFuture);
-                            futureRef.set(newFuture);
+                            
+                            synchronized (scheduleState) {
+                                if ( scheduleState.isScheduled() ) {
+                                    scheduleState.replaceFuture(scheduledFuture, newFuture);
+                                    futureRef.set(newFuture);
+                                }
+                            }
                         }
                     } else if ( shouldYield ) {
                         // Component itself didn't yield but there was no work to do, so the framework will choose
@@ -149,8 +154,13 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
                         if (scheduledFuture.cancel(false)) {
                             final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, NO_WORK_YIELD_NANOS, 
                                     connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
-                            scheduleState.replaceFuture(scheduledFuture, newFuture);
-                            futureRef.set(newFuture);
+                            
+                            synchronized (scheduleState) {
+                                if ( scheduleState.isScheduled() ) {
+                                    scheduleState.replaceFuture(scheduledFuture, newFuture);
+                                    futureRef.set(newFuture);
+                                }
+                            }
                         }
                     }
                 }


[26/29] incubator-nifi git commit: Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop

Posted by ma...@apache.org.
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/d939359b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d939359b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d939359b

Branch: refs/heads/develop
Commit: d939359bca5a2494756aa272829db677c7429e38
Parents: f1e74cc 57b5d58
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Feb 22 11:19:17 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Feb 22 11:19:17 2015 -0500

----------------------------------------------------------------------
 .../src/main/webapp/js/nf/canvas/nf-actions.js  | 57 +++++++++++-------
 .../webapp/js/nf/canvas/nf-canvas-header.js     | 62 +++++++++++---------
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    | 23 +-------
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js | 25 ++++++++
 .../main/webapp/js/nf/canvas/nf-context-menu.js |  7 +--
 5 files changed, 97 insertions(+), 77 deletions(-)
----------------------------------------------------------------------



[20/29] incubator-nifi git commit: NIFI-282: Allow Builder to be populated from SiteToSiteClientConfig object

Posted by ma...@apache.org.
NIFI-282: Allow Builder to be populated from SiteToSiteClientConfig object


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/7ab4392c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/7ab4392c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/7ab4392c

Branch: refs/heads/develop
Commit: 7ab4392c1571fffaea1f07fd7a17279122fa3fed
Parents: 46eebc7
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 17 10:46:29 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 17 10:46:29 2015 -0500

----------------------------------------------------------------------
 .../nifi/remote/client/SiteToSiteClient.java    | 23 ++++++++++++++++++++
 .../client/socket/TestSiteToSiteClient.java     |  2 +-
 2 files changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7ab4392c/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
index 5f84382..629032a 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -141,6 +141,29 @@ public interface SiteToSiteClient extends Closeable {
 		private long batchNanos;
 
 		/**
+		 * Populates the builder with values from the provided config
+		 * @param config
+		 * @return
+		 */
+		public Builder fromConfig(final SiteToSiteClientConfig config) {
+		    this.url = config.getUrl();
+		    this.timeoutNanos = config.getTimeout(TimeUnit.NANOSECONDS);
+		    this.penalizationNanos = config.getPenalizationPeriod(TimeUnit.NANOSECONDS);
+		    this.idleExpirationNanos = config.getIdleConnectionExpiration(TimeUnit.NANOSECONDS);
+		    this.sslContext = config.getSslContext();
+		    this.eventReporter = config.getEventReporter();
+		    this.peerPersistenceFile = config.getPeerPersistenceFile();
+		    this.useCompression = config.isUseCompression();
+		    this.portName = config.getPortName();
+		    this.portIdentifier = config.getPortIdentifier();
+		    this.batchCount = config.getPreferredBatchCount();
+		    this.batchSize = config.getPreferredBatchSize();
+		    this.batchNanos = config.getPreferredBatchDuration(TimeUnit.NANOSECONDS);
+		    
+		    return this;
+		}
+		
+		/**
 		 * Specifies the URL of the remote NiFi instance. If this URL points to the Cluster Manager of
 		 * a NiFi cluster, data transfer to and from nodes will be automatically load balanced across
 		 * the different nodes.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7ab4392c/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
index 0f48b03..b73e44d 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
@@ -36,7 +36,7 @@ import org.junit.Test;
 public class TestSiteToSiteClient {
 
     @Test
-    //@Ignore("For local testing only; not really a unit test but a manual test")
+    @Ignore("For local testing only; not really a unit test but a manual test")
     public void testReceive() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         


[25/29] incubator-nifi git commit: Merge branch 'develop' into nifi-site-to-site-client

Posted by ma...@apache.org.
Merge branch 'develop' into nifi-site-to-site-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/5291084d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5291084d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5291084d

Branch: refs/heads/develop
Commit: 5291084dc04c5fd328619e004dbd15569e90f70d
Parents: dbae98f f1e74cc
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Feb 22 11:18:26 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Feb 22 11:18:26 2015 -0500

----------------------------------------------------------------------
 README.md                                       |   6 +-
 nifi-site/src/includes/topbar.hbs               |   1 +
 nifi-site/src/pages/html/screencasts.hbs        |  35 ++++
 nifi/README.md                                  |   2 +-
 nifi/nifi-api/pom.xml                           |   1 -
 nifi/nifi-assembly/pom.xml                      |   1 -
 .../nifi-data-provenance-utils/pom.xml          |   1 -
 .../nifi-expression-language/pom.xml            |   1 -
 .../nifi-commons/nifi-flowfile-packager/pom.xml |   1 -
 nifi/nifi-commons/nifi-logging-utils/pom.xml    |   1 -
 .../nifi-processor-utilities/pom.xml            |   1 -
 nifi/nifi-commons/nifi-properties/pom.xml       |   1 -
 nifi/nifi-commons/nifi-security-utils/pom.xml   |   1 -
 nifi/nifi-commons/nifi-socket-utils/pom.xml     |   1 -
 nifi/nifi-commons/nifi-web-utils/pom.xml        |   1 -
 nifi/nifi-commons/nifi-write-ahead-log/pom.xml  |   1 -
 .../src/main/asciidoc/administration-guide.adoc | 139 +++++++++++++--
 nifi/nifi-docs/src/main/asciidoc/images/ncm.png | Bin 0 -> 339522 bytes
 nifi/nifi-docs/src/main/asciidoc/overview.adoc  |  20 +--
 nifi/nifi-mock/pom.xml                          |   1 -
 .../nifi-framework-nar/pom.xml                  |   1 -
 .../nifi-framework/nifi-administration/pom.xml  |   1 -
 .../nifi-cluster-protocol/pom.xml               |   1 -
 .../nifi-file-authorization-provider/pom.xml    |   1 -
 .../nifi-framework-core-api/pom.xml             |   1 -
 .../apache/nifi/controller/StandardFunnel.java  |   2 +-
 .../scheduling/QuartzSchedulingAgent.java       |  21 ++-
 .../controller/scheduling/ScheduleState.java    |  18 +-
 .../scheduling/TimerDrivenSchedulingAgent.java  | 115 ++++++++++--
 .../tasks/ContinuallyRunConnectableTask.java    |  32 ++--
 .../tasks/ContinuallyRunProcessorTask.java      |  32 ++--
 .../nifi-framework/nifi-security/pom.xml        |   1 -
 .../nifi-framework/nifi-site-to-site/pom.xml    |   1 -
 .../nifi-framework/nifi-user-actions/pom.xml    |   1 -
 .../partials/canvas/fill-color-dialog.jsp       |   6 +-
 .../nifi-web-ui/src/main/webapp/css/dialog.css  |   6 +-
 .../src/main/webapp/js/nf/canvas/nf-actions.js  |  39 +++--
 .../webapp/js/nf/canvas/nf-canvas-header.js     |  56 ++++--
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    |  15 +-
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    |  56 +++---
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js |   2 +-
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   |  25 ++-
 .../main/webapp/js/nf/canvas/nf-connectable.js  | 111 ++++++++----
 .../src/main/webapp/js/nf/canvas/nf-graph.js    |   6 +-
 .../nf-remote-process-group-configuration.js    |   3 +
 .../webapp/js/nf/cluster/nf-cluster-table.js    |   8 +-
 .../webapp/js/nf/counters/nf-counters-table.js  |   2 +-
 .../js/nf/provenance/nf-provenance-table.js     |   4 +-
 .../webapp/js/nf/summary/nf-summary-table.js    |  14 +-
 .../js/nf/templates/nf-templates-table.js       |   6 +-
 .../main/webapp/js/nf/users/nf-users-table.js   |   2 +-
 .../nifi-framework/pom.xml                      |   1 -
 .../nifi-framework-bundle/pom.xml               |   1 -
 .../nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml  |   1 -
 .../nifi-nar-bundles/nifi-hadoop-bundle/pom.xml |   1 -
 .../nifi-hadoop-libraries-nar/pom.xml           |   1 -
 .../nifi-hadoop-libraries-bundle/pom.xml        |   1 -
 nifi/nifi-nar-bundles/nifi-jetty-bundle/pom.xml |   2 -
 .../nifi-kafka-bundle/nifi-kafka-nar/pom.xml    |   1 -
 nifi/nifi-nar-bundles/nifi-kafka-bundle/pom.xml |   2 +-
 .../nifi-provenance-repository-nar/pom.xml      |   1 -
 .../nifi-provenance-repository-bundle/pom.xml   |   1 -
 .../nifi-standard-nar/pom.xml                   |   1 -
 .../nifi/processors/standard/EvaluateXPath.java |   2 +-
 .../apache/nifi/processors/standard/GetFTP.java |   5 +
 .../nifi/processors/standard/JmsConsumer.java   | 143 ++++++++-------
 .../apache/nifi/processors/standard/PutFTP.java |   7 +-
 .../apache/nifi/processors/standard/PutJMS.java |  11 +-
 .../processors/standard/util/FTPTransfer.java   |  48 ++++-
 .../standard/util/JmsProcessingSummary.java     |  83 +++++++++
 .../processors/standard/util/JmsProperties.java |   3 +-
 .../standard/util/SocksProxySocketFactory.java  |  69 ++++++++
 .../processors/standard/TestJmsConsumer.java    | 173 +++++++++++++++++++
 .../nifi-standard-bundle/pom.xml                |   1 -
 .../nifi-distributed-cache-services-nar/pom.xml |   1 -
 .../pom.xml                                     |   1 -
 .../nifi-load-distribution-service-api/pom.xml  |   1 -
 .../nifi-ssl-context-nar/pom.xml                |   1 -
 .../nifi-ssl-context-bundle/pom.xml             |   1 -
 .../nifi-standard-services-api-nar/pom.xml      |   1 -
 .../nifi-standard-services/pom.xml              |   1 -
 .../nifi-update-attribute-nar/pom.xml           |   2 -
 .../nifi-update-attribute-bundle/pom.xml        |   3 -
 nifi/nifi-nar-bundles/pom.xml                   |   3 -
 84 files changed, 1071 insertions(+), 308 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5291084d/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5291084d/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
----------------------------------------------------------------------


[27/29] incubator-nifi git commit: Merge branch 'nifi-site-to-site-client' into develop

Posted by ma...@apache.org.
Merge branch 'nifi-site-to-site-client' into develop


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/40e1fcbb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/40e1fcbb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/40e1fcbb

Branch: refs/heads/develop
Commit: 40e1fcbb982520851c4591970e70b03b2ab9109e
Parents: d939359 5291084
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Feb 22 11:19:24 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Feb 22 11:19:24 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/remote/RemoteDestination.java   |  53 +
 .../nifi-site-to-site-client/pom.xml            |  45 +
 .../remote/AbstractCommunicationsSession.java   |  54 ++
 .../org/apache/nifi/remote/Communicant.java     |  47 +
 .../main/java/org/apache/nifi/remote/Peer.java  | 151 +++
 .../org/apache/nifi/remote/PeerDescription.java |  79 ++
 .../java/org/apache/nifi/remote/PeerStatus.java |  60 ++
 .../nifi/remote/RemoteResourceInitiator.java    |  73 ++
 .../org/apache/nifi/remote/Transaction.java     | 214 ++++
 .../nifi/remote/TransactionCompletion.java      |  63 ++
 .../apache/nifi/remote/TransferDirection.java   |  34 +
 .../nifi/remote/VersionedRemoteResource.java    |  24 +
 .../nifi/remote/client/SiteToSiteClient.java    | 519 ++++++++++
 .../remote/client/SiteToSiteClientConfig.java   | 124 +++
 .../client/socket/EndpointConnection.java       |  54 ++
 .../client/socket/EndpointConnectionPool.java   | 970 +++++++++++++++++++
 .../nifi/remote/client/socket/SocketClient.java | 218 +++++
 .../remote/cluster/AdaptedNodeInformation.java  |  66 ++
 .../remote/cluster/ClusterNodeInformation.java  |  67 ++
 .../nifi/remote/cluster/NodeInformation.java    |  98 ++
 .../remote/cluster/NodeInformationAdapter.java  |  41 +
 .../apache/nifi/remote/codec/FlowFileCodec.java |  71 ++
 .../remote/codec/StandardFlowFileCodec.java     | 129 +++
 .../remote/exception/HandshakeException.java    |  37 +
 .../exception/PortNotRunningException.java      |  30 +
 .../remote/exception/ProtocolException.java     |  40 +
 .../remote/exception/UnknownPortException.java  |  29 +
 .../SocketChannelCommunicationsSession.java     |  90 ++
 .../remote/io/socket/SocketChannelInput.java    |  71 ++
 .../remote/io/socket/SocketChannelOutput.java   |  58 ++
 .../SSLSocketChannelCommunicationsSession.java  |  93 ++
 .../io/socket/ssl/SSLSocketChannelInput.java    |  55 ++
 .../io/socket/ssl/SSLSocketChannelOutput.java   |  44 +
 .../nifi/remote/protocol/ClientProtocol.java    |  86 ++
 .../remote/protocol/CommunicationsInput.java    |  33 +
 .../remote/protocol/CommunicationsOutput.java   |  27 +
 .../remote/protocol/CommunicationsSession.java  |  64 ++
 .../apache/nifi/remote/protocol/DataPacket.java |  45 +
 .../nifi/remote/protocol/RequestType.java       |  43 +
 .../protocol/socket/HandshakeProperty.java      |  61 ++
 .../nifi/remote/protocol/socket/Response.java   |  51 +
 .../remote/protocol/socket/ResponseCode.java    | 153 +++
 .../protocol/socket/SocketClientProtocol.java   | 438 +++++++++
 .../socket/SocketClientTransaction.java         | 399 ++++++++
 .../SocketClientTransactionCompletion.java      |  57 ++
 .../nifi/remote/util/NiFiRestApiUtil.java       |  98 ++
 .../nifi/remote/util/PeerStatusCache.java       |  43 +
 .../nifi/remote/util/StandardDataPacket.java    |  50 +
 .../socket/TestEndpointConnectionStatePool.java |  95 ++
 .../client/socket/TestSiteToSiteClient.java     | 105 ++
 .../io/socket/SocketChannelInputStream.java     |  10 +
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  10 +
 .../socket/ssl/SSLSocketChannelInputStream.java |   4 +
 .../nifi/stream/io/LimitingInputStream.java     | 111 +++
 .../stream/io/MinimumLengthInputStream.java     |  93 ++
 nifi/nifi-commons/pom.xml                       |   1 +
 nifi/nifi-external/README.md                    |  19 +
 nifi/nifi-external/nifi-spark-receiver/pom.xml  |  37 +
 .../org/apache/nifi/spark/NiFiDataPacket.java   |  40 +
 .../org/apache/nifi/spark/NiFiReceiver.java     | 198 ++++
 nifi/nifi-external/pom.xml                      |  29 +
 .../nifi-framework/nifi-cluster/pom.xml         |   4 +
 .../nifi/cluster/manager/ClusterManager.java    |  18 +-
 .../cluster/manager/impl/WebClusterManager.java |   9 +-
 .../nifi-framework-core-api/pom.xml             |   4 +
 .../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  |  39 +-
 .../main/java/org/apache/nifi/remote/Peer.java  | 107 --
 .../java/org/apache/nifi/remote/PeerStatus.java |  72 --
 .../org/apache/nifi/remote/RemoteGroupPort.java |  22 +-
 .../apache/nifi/remote/TransferDirection.java   |  23 -
 .../nifi/remote/VersionedRemoteResource.java    |  24 -
 .../nifi/remote/cluster/NodeInformant.java      |  22 +
 .../apache/nifi/remote/codec/FlowFileCodec.java |  79 --
 .../remote/exception/HandshakeException.java    |  30 -
 .../exception/PortNotRunningException.java      |  26 -
 .../remote/exception/ProtocolException.java     |  34 -
 .../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-framework/nifi-framework-core/pom.xml  |   4 +
 .../apache/nifi/controller/FlowController.java  |   5 +-
 .../org/apache/nifi/remote/RemoteNiFiUtils.java | 216 +++++
 .../nifi/remote/StandardRemoteProcessGroup.java | 309 +-----
 .../nifi-framework/nifi-site-to-site/.gitignore |   1 +
 .../util/RemoteProcessGroupUtils.class          | Bin 0 -> 9526 bytes
 .../remote/AbstractCommunicationsSession.class  | Bin 0 -> 2308 bytes
 .../nifi/remote/RemoteResourceFactory.class     | Bin 0 -> 8707 bytes
 .../nifi/remote/RemoteResourceManager.class     | Bin 0 -> 6898 bytes
 .../apache/nifi/remote/RemoteSiteListener.class | Bin 0 -> 841 bytes
 .../nifi/remote/SocketRemoteSiteListener.class  | Bin 0 -> 8448 bytes
 ...emoteGroupPort$EndpointConnectionState.class | Bin 0 -> 5427 bytes
 .../nifi/remote/StandardRemoteGroupPort.class   | Bin 0 -> 10677 bytes
 .../StandardRootGroupPort$FlowFileRequest.class | Bin 0 -> 5836 bytes
 ...StandardRootGroupPort$ProcessingResult.class | Bin 0 -> 5032 bytes
 ...upPort$StandardPortAuthorizationResult.class | Bin 0 -> 5159 bytes
 .../nifi/remote/StandardRootGroupPort.class     | Bin 0 -> 9700 bytes
 .../remote/codec/StandardFlowFileCodec.class    | Bin 0 -> 8538 bytes
 .../exception/UnsupportedCodecException.class   | Bin 0 -> 1057 bytes
 .../SocketChannelCommunicationsSession.class    | Bin 0 -> 3735 bytes
 .../remote/io/socket/SocketChannelInput.class   | Bin 0 -> 4008 bytes
 .../remote/io/socket/SocketChannelOutput.class  | Bin 0 -> 3741 bytes
 .../SSLSocketChannelCommunicationsSession.class | Bin 0 -> 4611 bytes
 .../io/socket/ssl/SSLSocketChannelInput.class   | Bin 0 -> 3127 bytes
 .../io/socket/ssl/SSLSocketChannelOutput.class  | Bin 0 -> 2587 bytes
 .../socket/ClusterManagerServerProtocol.class   | Bin 0 -> 10540 bytes
 .../protocol/socket/HandshakeProperty.class     | Bin 0 -> 917 bytes
 .../nifi/remote/protocol/socket/Response.class  | Bin 0 -> 2674 bytes
 .../remote/protocol/socket/ResponseCode.class   | Bin 0 -> 6889 bytes
 .../protocol/socket/SocketClientProtocol.class  | Bin 0 -> 8965 bytes
 .../socket/SocketFlowFileServerProtocol.class   | Bin 0 -> 8806 bytes
 .../remote/TestStandardRemoteGroupPort.class    | Bin 0 -> 5974 bytes
 .../nifi-framework/nifi-site-to-site/pom.xml    |   4 +
 .../remote/AbstractCommunicationsSession.java   |  54 --
 .../nifi/remote/RemoteResourceFactory.java      |  50 +-
 .../nifi/remote/SocketRemoteSiteListener.java   |  16 +-
 .../nifi/remote/StandardRemoteGroupPort.java    | 685 ++++---------
 .../remote/codec/StandardFlowFileCodec.java     | 169 ----
 .../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    | 195 ++--
 .../remote/TestStandardRemoteGroupPort.java     |  97 --
 .../main/webapp/js/nf/canvas/nf-context-menu.js |   2 +-
 nifi/pom.xml                                    |   6 +
 141 files changed, 6833 insertions(+), 3397 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/40e1fcbb/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
----------------------------------------------------------------------


[02/29] incubator-nifi git commit: NIFI-282: Bug fixes; documentation improvements; removed code marked as 'FOR TESTING'

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 2c1b085..db0aeb7 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -18,21 +18,9 @@ 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 +37,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;
@@ -64,7 +49,6 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessScheduler;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.exception.CommunicationsException;
-import org.apache.nifi.controller.util.RemoteProcessGroupUtils;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.events.BulletinFactory;
 import org.apache.nifi.events.EventReporter;
@@ -72,16 +56,8 @@ 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.remote.util.RemoteNiFiUtils;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.util.FormatUtils;
@@ -108,7 +84,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
 
     public static final String CONTROLLER_URI_PATH = "/controller";
     public static final String ROOT_GROUP_STATUS_URI_PATH = "/controller/process-groups/root/status";
-    public static final long LISTENING_PORT_REFRESH_MILLIS = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
 
     // status codes
     public static final int OK_STATUS_CODE = Status.OK.getStatusCode();
@@ -150,15 +125,12 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
 
     private ProcessGroupCounts counts = new ProcessGroupCounts(0, 0, 0, 0, 0, 0, 0, 0);
     private Long refreshContentsTimestamp = null;
-    private Integer listeningPort;
-    private long listeningPortRetrievalTime = 0L;
     private Boolean destinationSecure;
+    private Integer listeningPort;
 
     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,
@@ -200,72 +172,13 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
             }
         };
 
-        final Runnable socketCleanup = new Runnable() {
-            @Override
-            public void run() {
-                final Set<StandardRemoteGroupPort> ports = new HashSet<>();
-                readLock.lock();
-                try {
-                    ports.addAll(inputPorts.values());
-                    ports.addAll(outputPorts.values());
-                } finally {
-                    readLock.unlock();
-                }
-
-                for (final StandardRemoteGroupPort port : ports) {
-                    port.cleanupSockets();
-                }
-            }
-        };
-
-        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);
-        }
-
-        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);
-                    }
-                }
-            }
-        };
-
+        endpointConnectionPool = new EndpointConnectionStatePool(getTargetUri().toString(), getCommunicationsTimeout(TimeUnit.MILLISECONDS), 
+        		sslContext, eventReporter, getPeerPersistenceFile());
+        
         final Runnable checkAuthorizations = new InitializationTask();
 
         backgroundThreadExecutor = new FlowEngine(1, "Remote Process Group " + id + ": " + targetUri);
         backgroundThreadExecutor.scheduleWithFixedDelay(checkAuthorizations, 0L, 30L, TimeUnit.SECONDS);
-        backgroundThreadExecutor.scheduleWithFixedDelay(refreshPeers, 0, 5, TimeUnit.SECONDS);
-        backgroundThreadExecutor.scheduleWithFixedDelay(socketCleanup, 10L, 10L, TimeUnit.SECONDS);
     }
 
     @Override
@@ -287,6 +200,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     @Override
     public void shutdown() {
         backgroundThreadExecutor.shutdown();
+        endpointConnectionPool.shutdown();
     }
     
     @Override
@@ -858,7 +772,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
             return;
         }
 
-        final RemoteProcessGroupUtils utils = new RemoteProcessGroupUtils(isWebApiSecure() ? sslContext : null);
+        final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
         final String uriVal = apiUri.toString() + CONTROLLER_URI_PATH;
         URI uri;
         try {
@@ -998,39 +912,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         return descriptor;
     }
 
-    /**
-     * @return the port that the remote instance is listening on for
-     * site-to-site communication, or <code>null</code> if the remote instance
-     * is not configured to allow site-to-site communications.
-     *
-     * @throws IOException if unable to communicate with the remote instance
-     */
-    @Override
-    public Integer getListeningPort() throws IOException {
-        Integer listeningPort;
-        readLock.lock();
-        try {
-            listeningPort = this.listeningPort;
-            if (listeningPort != null && this.listeningPortRetrievalTime > System.currentTimeMillis() - LISTENING_PORT_REFRESH_MILLIS) {
-                return listeningPort;
-            }
-        } finally {
-            readLock.unlock();
-        }
-
-        final RemoteProcessGroupUtils utils = new RemoteProcessGroupUtils(isWebApiSecure() ? sslContext : null);
-        listeningPort = utils.getRemoteListeningPort(apiUri.toString(), getCommunicationsTimeout(TimeUnit.MILLISECONDS));
-        writeLock.lock();
-        try {
-            this.listeningPort = listeningPort;
-            this.listeningPortRetrievalTime = System.currentTimeMillis();
-        } finally {
-            writeLock.unlock();
-        }
-
-        return listeningPort;
-    }
-
     @Override
     public boolean isTransmitting() {
         return transmitting.get();
@@ -1255,52 +1136,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() {
@@ -1312,7 +1147,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         @Override
         public void run() {
             try {
-                final RemoteProcessGroupUtils utils = new RemoteProcessGroupUtils(isWebApiSecure() ? sslContext : null);
+                final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
                 final ClientResponse response = utils.get(new URI(apiUri + CONTROLLER_URI_PATH), getCommunicationsTimeout(TimeUnit.MILLISECONDS));
                 
                 final int statusCode = response.getStatus();
@@ -1385,6 +1220,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     public String getYieldDuration() {
         return yieldDuration;
     }
+    
+    @Override
+    public EndpointConnectionStatePool getConnectionPool() {
+        return endpointConnectionPool;
+    }
 
     @Override
     public void verifyCanDelete() {
@@ -1487,135 +1327,9 @@ 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();
-    }
-
-    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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore
index ea8c4bf..d9d66d8 100755
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/.gitignore
@@ -1 +1,2 @@
 /target
+/bin/

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class
new file mode 100644
index 0000000..a6951d4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class
new file mode 100644
index 0000000..2e868ea
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class
new file mode 100644
index 0000000..9c6e821
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class
new file mode 100644
index 0000000..bb3fc77
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class
new file mode 100644
index 0000000..a56b5ba
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class
new file mode 100644
index 0000000..9780f75
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class
new file mode 100644
index 0000000..f184d64
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class
new file mode 100644
index 0000000..0740bb4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class
new file mode 100644
index 0000000..d517458
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class
new file mode 100644
index 0000000..1cf5ceb
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class
new file mode 100644
index 0000000..3ad7542
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class
new file mode 100644
index 0000000..4db4735
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class
new file mode 100644
index 0000000..e49ffe8
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class
new file mode 100644
index 0000000..39dd49a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class
new file mode 100644
index 0000000..b415421
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class
new file mode 100644
index 0000000..551097e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class
new file mode 100644
index 0000000..6913767
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class
new file mode 100644
index 0000000..f6e9f20
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class
new file mode 100644
index 0000000..41fe366
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class
new file mode 100644
index 0000000..9b9cdc0
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class
new file mode 100644
index 0000000..2ef1c39
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class
new file mode 100644
index 0000000..fad8245
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class
new file mode 100644
index 0000000..27ec8d3
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class
new file mode 100644
index 0000000..4673aec
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class
new file mode 100644
index 0000000..ac7e1b9
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class
new file mode 100644
index 0000000..933ef4d
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class
new file mode 100644
index 0000000..0e77276
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
index f989f66..08bf590 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
@@ -60,6 +60,10 @@
             <artifactId>nifi-utils</artifactId>
         </dependency>
         <dependency>
+        	<groupId>org.apache.nifi</groupId>
+        	<artifactId>nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.java
deleted file mode 100644
index 4babb92..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
index 49d3c3c..2b27de2 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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();
@@ -96,6 +56,14 @@ public class RemoteResourceFactory {
         }
 	}
 	
+	public static void rejectCodecNegotiation(final DataInputStream dis, final DataOutputStream dos, final String explanation) throws IOException {
+		dis.readUTF();	// read codec name
+		dis.readInt();	// read codec version
+		
+		dos.write(ABORT);
+		dos.writeUTF(explanation);
+		dos.flush();
+	}
 	
 	@SuppressWarnings("unchecked")
     public static <T extends ClientProtocol> T receiveClientProtocolNegotiation(final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
index ec169ad..3295956 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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;
@@ -122,6 +122,9 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
                     }
                     LOG.trace("Got connection");
                     
+                    if ( stopped.get() ) {
+                        return;
+                    }
                     final Socket socket = acceptedSocket;
                     final SocketChannel socketChannel = socket.getChannel();
                     final Thread thread = new Thread(new Runnable() {
@@ -198,7 +201,7 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
                             	protocol.setRootProcessGroup(rootGroup.get());
                           	    protocol.setNodeInformant(nodeInformant);
                             	
-                            	peer = new Peer(commsSession, peerUri);
+                            	peer = new Peer(commsSession, peerUri, "nifi://localhost:" + getPort());
                             	LOG.debug("Handshaking....");
                             	protocol.handshake(peer);
                             	

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index b0d88d4..a51cdba 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -16,90 +16,59 @@
  */
 package org.apache.nifi.remote;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.channels.SocketChannel;
 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.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 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.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.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.ClientProtocol;
 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.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 +82,10 @@ 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");
+        
+        connectionStatePool = remoteGroup.getConnectionPool();
     }
     
     @Override
@@ -133,25 +104,10 @@ 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();
         }
@@ -170,35 +126,6 @@ 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);
-        }
-    }
-    
-    
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession session) {
         if ( !remoteGroup.isTransmitting() ) {
@@ -212,137 +139,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(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 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 +197,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 +211,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 +232,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 +260,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 +269,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 +354,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;


[16/29] incubator-nifi git commit: NIFI-282: Refactored to remove Jersey client from dependencies; made site-to-site config serializable; allowed SiteToSiteClient.Builder to build a SiteToSiteClientConfig without building the client itself.

Posted by ma...@apache.org.
NIFI-282: Refactored to remove Jersey client from dependencies; made site-to-site config serializable; allowed SiteToSiteClient.Builder to build a SiteToSiteClientConfig without building the client itself.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/e16fc797
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/e16fc797
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/e16fc797

Branch: refs/heads/develop
Commit: e16fc7972c24a04d8212e26f66fdcb6e940ffe86
Parents: 8f0402f
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 16 14:18:24 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 16 14:18:24 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/remote/RemoteDestination.java   |   8 +-
 .../nifi-site-to-site-client/pom.xml            |  80 +++----
 .../nifi/remote/client/SiteToSiteClient.java    | 156 ++++++++------
 .../remote/client/SiteToSiteClientConfig.java   |   3 +-
 .../client/socket/EndpointConnectionPool.java   |  83 +++++--
 .../nifi/remote/client/socket/SocketClient.java |  10 +-
 .../nifi/remote/util/NiFiRestApiUtil.java       |  98 +++++++++
 .../nifi/remote/util/RemoteNiFiUtils.java       | 216 -------------------
 .../client/socket/TestSiteToSiteClient.java     |  17 +-
 .../org/apache/nifi/remote/RemoteNiFiUtils.java | 216 +++++++++++++++++++
 .../nifi/remote/StandardRemoteProcessGroup.java |   1 -
 11 files changed, 533 insertions(+), 355 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/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
index f718581..508ab37 100644
--- 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
@@ -29,7 +29,13 @@ public interface RemoteDestination {
      * @return
      */
 	String getIdentifier();
-	
+
+	/**
+	 * Returns the human-readable name of the remote destination
+	 * @return
+	 */
+	String getName();
+
 	/**
 	 * Returns the amount of time that system should pause sending to a particular node if unable to 
 	 * send data to or receive data from this endpoint

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml
index 3fc00a2..0d21a3d 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml
+++ b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml
@@ -1,43 +1,45 @@
 <?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</artifactId>
-    <version>0.0.2-incubating-SNAPSHOT</version>
-  </parent>
-  
-  <artifactId>nifi-site-to-site-client</artifactId>
-  
-  <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>com.sun.jersey</groupId>
-		<artifactId>jersey-client</artifactId>
-	</dependency>
-	<dependency>
+<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-client-dto</artifactId>
+		<artifactId>nifi-commons</artifactId>
 		<version>0.0.2-incubating-SNAPSHOT</version>
-	</dependency>
-  	<dependency>
-  		<groupId>org.apache.nifi</groupId>
-  		<artifactId>nifi-web-utils</artifactId>
-  	</dependency>
-  	
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
+	</parent>
+
+	<artifactId>nifi-site-to-site-client</artifactId>
+
+	<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>com.sun.jersey</groupId> <artifactId>jersey-client</artifactId> 
+			</dependency> <dependency> <groupId>org.apache.nifi</groupId> <artifactId>nifi-web-utils</artifactId> 
+			</dependency> -->
+		<dependency>
+			<groupId>org.codehaus.jackson</groupId>
+			<artifactId>jackson-mapper-asl</artifactId>
+			<version>1.9.13</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-client-dto</artifactId>
+			<version>0.0.2-incubating-SNAPSHOT</version>
+		</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/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
index 0591b5a..5f84382 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -19,6 +19,7 @@ package org.apache.nifi.remote.client;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.concurrent.TimeUnit;
 
 import javax.net.ssl.SSLContext;
@@ -122,8 +123,10 @@ public interface SiteToSiteClient extends Closeable {
 	 * and a new client created. 
 	 * </p>
 	 */
-	public static class Builder {
-		private String url;
+	public static class Builder implements Serializable {
+        private static final long serialVersionUID = -4954962284343090219L;
+        
+        private String url;
 		private long timeoutNanos = TimeUnit.SECONDS.toNanos(30);
 		private long penalizationNanos = TimeUnit.SECONDS.toNanos(3);
 		private long idleExpirationNanos = TimeUnit.SECONDS.toNanos(30L);
@@ -309,10 +312,89 @@ public interface SiteToSiteClient extends Closeable {
 		    return this;
 		}
 		
+		/**
+		 * Returns a {@link SiteToSiteClientConfig} for the configured values but does not create a SiteToSiteClient
+		 * @return
+		 */
+		public SiteToSiteClientConfig buildConfig() {
+		    final SiteToSiteClientConfig config = new SiteToSiteClientConfig() {
+                private static final long serialVersionUID = 1323119754841633818L;
+
+                @Override
+                public boolean isUseCompression() {
+                    return Builder.this.isUseCompression();
+                }
+                
+                @Override
+                public String getUrl() {
+                    return Builder.this.getUrl();
+                }
+                
+                @Override
+                public long getTimeout(final TimeUnit timeUnit) {
+                    return Builder.this.getTimeout(timeUnit);
+                }
+                
+                @Override
+                public long getIdleConnectionExpiration(final TimeUnit timeUnit) {
+                    return Builder.this.getIdleConnectionExpiration(timeUnit);
+                }
+                
+                @Override
+                public SSLContext getSslContext() {
+                    return Builder.this.getSslContext();
+                }
+                
+                @Override
+                public String getPortName() {
+                    return Builder.this.getPortName();
+                }
+                
+                @Override
+                public String getPortIdentifier() {
+                    return Builder.this.getPortIdentifier();
+                }
+                
+                @Override
+                public long getPenalizationPeriod(final TimeUnit timeUnit) {
+                    return Builder.this.getPenalizationPeriod(timeUnit);
+                }
+                
+                @Override
+                public File getPeerPersistenceFile() {
+                    return Builder.this.getPeerPersistenceFile();
+                }
+                
+                @Override
+                public EventReporter getEventReporter() {
+                    return Builder.this.getEventReporter();
+                }
+
+                @Override
+                public long getPreferredBatchDuration(final TimeUnit timeUnit) {
+                    return timeUnit.convert(Builder.this.batchNanos, TimeUnit.NANOSECONDS);
+                }
+                
+                @Override
+                public long getPreferredBatchSize() {
+                    return Builder.this.batchSize;
+                }
+                
+                @Override
+                public int getPreferredBatchCount() {
+                    return Builder.this.batchCount;
+                }
+            };
+            
+            return config;
+		}
 		
 		/**
 		 * Builds a new SiteToSiteClient that can be used to send and receive data with remote instances of NiFi
 		 * @return
+		 * 
+		 * @throws IllegalStateException if either the url is not set or neither the port name nor port identifier
+		 * is set.
 		 */
 		public SiteToSiteClient build() {
 			if ( url == null ) {
@@ -323,75 +405,7 @@ public interface SiteToSiteClient extends Closeable {
 				throw new IllegalStateException("Must specify either Port Name or Port Identifier to builder Site-to-Site client");
 			}
 			
-			final SiteToSiteClientConfig config = new SiteToSiteClientConfig() {
-				
-				@Override
-				public boolean isUseCompression() {
-					return Builder.this.isUseCompression();
-				}
-				
-				@Override
-				public String getUrl() {
-					return Builder.this.getUrl();
-				}
-				
-				@Override
-				public long getTimeout(final TimeUnit timeUnit) {
-					return Builder.this.getTimeout(timeUnit);
-				}
-				
-				@Override
-				public long getIdleConnectionExpiration(final TimeUnit timeUnit) {
-				    return Builder.this.getIdleConnectionExpiration(timeUnit);
-				}
-				
-				@Override
-				public SSLContext getSslContext() {
-					return Builder.this.getSslContext();
-				}
-				
-				@Override
-				public String getPortName() {
-					return Builder.this.getPortName();
-				}
-				
-				@Override
-				public String getPortIdentifier() {
-					return Builder.this.getPortIdentifier();
-				}
-				
-				@Override
-				public long getPenalizationPeriod(final TimeUnit timeUnit) {
-					return Builder.this.getPenalizationPeriod(timeUnit);
-				}
-				
-				@Override
-				public File getPeerPersistenceFile() {
-					return Builder.this.getPeerPersistenceFile();
-				}
-				
-				@Override
-				public EventReporter getEventReporter() {
-					return Builder.this.getEventReporter();
-				}
-
-		        @Override
-		        public long getPreferredBatchDuration(final TimeUnit timeUnit) {
-		            return timeUnit.convert(Builder.this.batchNanos, TimeUnit.NANOSECONDS);
-		        }
-		        
-		        @Override
-		        public long getPreferredBatchSize() {
-		            return Builder.this.batchSize;
-		        }
-		        
-		        @Override
-		        public int getPreferredBatchCount() {
-		            return Builder.this.batchCount;
-		        }
-			};
-			
-			return new SocketClient(config);
+			return new SocketClient(buildConfig());
 		}
 
 		/**

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
index d03ab3c..5e7fbe8 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.remote.client;
 
 import java.io.File;
+import java.io.Serializable;
 import java.util.concurrent.TimeUnit;
 
 import javax.net.ssl.SSLContext;
@@ -24,7 +25,7 @@ import javax.net.ssl.SSLContext;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.remote.protocol.DataPacket;
 
-public interface SiteToSiteClientConfig {
+public interface SiteToSiteClientConfig extends Serializable {
 
 	/**
 	 * Returns the configured URL for the remote NiFi instance

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index c0e4761..f9a8a38 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -79,8 +79,8 @@ 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.NiFiRestApiUtil;
 import org.apache.nifi.remote.util.PeerStatusCache;
-import org.apache.nifi.remote.util.RemoteNiFiUtils;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.apache.nifi.web.api.dto.ControllerDTO;
@@ -201,6 +201,17 @@ public class EndpointConnectionPool {
     	}, 5, 5, TimeUnit.SECONDS);
     }
     
+    private String getPortIdentifier(final TransferDirection transferDirection) throws IOException {
+        if ( remoteDestination.getIdentifier() != null ) {
+            return remoteDestination.getIdentifier();
+        }
+        
+        if ( transferDirection == TransferDirection.RECEIVE ) {
+            return getOutputPortIdentifier(remoteDestination.getName());
+        } else {
+            return getInputPortIdentifier(remoteDestination.getName());
+        }
+    }
     
     public EndpointConnection getEndpointConnection(final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
         return getEndpointConnection(direction, null);
@@ -222,14 +233,15 @@ public class EndpointConnectionPool {
             do {
                 connection = connectionQueue.poll();
                 logger.debug("{} Connection State for {} = {}", this, clusterUrl, connection);
+                final String portId = getPortIdentifier(direction);
                 
                 if ( connection == null && !addBack.isEmpty() ) {
                     // all available connections have been penalized.
-                    logger.debug("{} all Connections for {} are penalized; returning no Connection", this, remoteDestination.getIdentifier());
+                    logger.debug("{} all Connections for {} are penalized; returning no Connection", this, portId);
                     return null;
                 }
                 
-                if ( connection != null && connection.getPeer().isPenalized(remoteDestination.getIdentifier()) ) {
+                if ( connection != null && connection.getPeer().isPenalized(portId) ) {
                     // we have a connection, but it's penalized. We want to add it back to the queue
                     // when we've found one to use.
                     addBack.add(connection);
@@ -238,9 +250,9 @@ public class EndpointConnectionPool {
                 
                 // if we can't get an existing Connection, create one
                 if ( connection == null ) {
-                    logger.debug("{} No Connection available for Port {}; creating new Connection", this, remoteDestination.getIdentifier());
+                    logger.debug("{} No Connection available for Port {}; creating new Connection", this, portId);
                     protocol = new SocketClientProtocol();
-                    protocol.setDestination(remoteDestination);
+                    protocol.setDestination(new IdEnrichedRemoteDestination(remoteDestination, portId));
 
                     logger.debug("{} getting next peer status", this);
                     final PeerStatus peerStatus = getNextPeerStatus(direction);
@@ -249,11 +261,12 @@ public class EndpointConnectionPool {
                         return null;
                     }
 
+                    final long penalizationMillis = remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS);
                     try {
                         logger.debug("{} Establishing site-to-site connection with {}", this, peerStatus);
                         commsSession = establishSiteToSiteConnection(peerStatus);
                     } catch (final IOException ioe) {
-                        penalize(peerStatus, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+                        penalize(peerStatus, penalizationMillis);
                         throw ioe;
                     }
                     
@@ -289,17 +302,17 @@ public class EndpointConnectionPool {
                         // 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));
+                            penalize(peer, penalizationMillis);
                             connectionQueue.offer(connection);
                             continue;
                         } else if ( protocol.isPortInvalid() ) {
-                        	penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+                        	penalize(peer, penalizationMillis);
                         	cleanup(protocol, peer);
-                        	throw new PortNotRunningException(peer.toString() + " indicates that port " + remoteDestination.getIdentifier() + " is not running");
+                        	throw new PortNotRunningException(peer.toString() + " indicates that port " + portId + " is not running");
                         } else if ( protocol.isPortUnknown() ) {
-                        	penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+                        	penalize(peer, penalizationMillis);
                         	cleanup(protocol, peer);
-                        	throw new UnknownPortException(peer.toString() + " indicates that port " + remoteDestination.getIdentifier() + " is not known");
+                        	throw new UnknownPortException(peer.toString() + " indicates that port " + portId + " is not known");
                         }
                         
                         // negotiate the FlowFileCodec to use
@@ -309,7 +322,7 @@ public class EndpointConnectionPool {
                     } catch (final PortNotRunningException | UnknownPortException e) {
                     	throw e;
                     } catch (final Exception e) {
-                        penalize(peer, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+                        penalize(peer, penalizationMillis);
                         cleanup(protocol, peer);
                         
                         final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? clusterUrl : peer, e.toString());
@@ -539,7 +552,16 @@ public class EndpointConnectionPool {
 
         clientProtocol.setTimeout(commsTimeout);
         if (clientProtocol.getVersionNegotiator().getVersion() < 5) {
-            clientProtocol.handshake(peer, remoteDestination.getIdentifier());
+            String portId = getPortIdentifier(TransferDirection.RECEIVE);
+            if ( portId == null ) {
+                portId = getPortIdentifier(TransferDirection.SEND);
+            }
+            
+            if ( portId == null ) {
+                peer.close();
+                throw new IOException("Failed to determine the identifier of port " + remoteDestination.getName());
+            }
+            clientProtocol.handshake(peer, portId);
         } else {
             clientProtocol.handshake(peer, null);
         }
@@ -818,8 +840,8 @@ public class EndpointConnectionPool {
     
     private ControllerDTO refreshRemoteInfo() throws IOException {
     	final boolean webInterfaceSecure = clusterUrl.toString().startsWith("https");
-        final RemoteNiFiUtils utils = new RemoteNiFiUtils(webInterfaceSecure ? sslContext : null);
-		final ControllerDTO controller = utils.getController(URI.create(apiUri + "/controller"), commsTimeout);
+        final NiFiRestApiUtil utils = new NiFiRestApiUtil(webInterfaceSecure ? sslContext : null);
+		final ControllerDTO controller = utils.getController(apiUri + "/controller", commsTimeout);
         
         remoteInfoWriteLock.lock();
         try {
@@ -898,4 +920,35 @@ public class EndpointConnectionPool {
         
         return isSecure;
     }
+    
+    
+    private class IdEnrichedRemoteDestination implements RemoteDestination {
+        private final RemoteDestination original;
+        private final String identifier;
+        
+        public IdEnrichedRemoteDestination(final RemoteDestination original, final String identifier) {
+            this.original = original;
+            this.identifier = identifier;
+        }
+
+        @Override
+        public String getIdentifier() {
+            return identifier;
+        }
+
+        @Override
+        public String getName() {
+            return original.getName();
+        }
+
+        @Override
+        public long getYieldPeriod(final TimeUnit timeUnit) {
+            return original.getYieldPeriod(timeUnit);
+        }
+
+        @Override
+        public boolean isUseCompression() {
+            return original.isUseCompression();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
index 016e67f..c11c2ab 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -43,7 +43,8 @@ public class SocketClient implements SiteToSiteClient {
 	private volatile boolean closed = false;
 	
 	public SocketClient(final SiteToSiteClientConfig config) {
-		pool = new EndpointConnectionPool(config.getUrl(), createRemoteDestination(config.getPortIdentifier()), 
+		pool = new EndpointConnectionPool(config.getUrl(), 
+		        createRemoteDestination(config.getPortIdentifier(), config.getPortName()),
 		        (int) config.getTimeout(TimeUnit.MILLISECONDS),
 		        (int) config.getIdleConnectionExpiration(TimeUnit.MILLISECONDS),
 				config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile());
@@ -88,12 +89,17 @@ public class SocketClient implements SiteToSiteClient {
 	}
 	
 	
-	private RemoteDestination createRemoteDestination(final String portId) {
+	private RemoteDestination createRemoteDestination(final String portId, final String portName) {
 	    return new RemoteDestination() {
             @Override
             public String getIdentifier() {
                 return portId;
             }
+            
+            @Override
+            public String getName() {
+                return portName;
+            }
 
             @Override
             public long getYieldPeriod(final TimeUnit timeUnit) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/NiFiRestApiUtil.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/NiFiRestApiUtil.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/NiFiRestApiUtil.java
new file mode 100644
index 0000000..10352ec
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/NiFiRestApiUtil.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.util;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSession;
+
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.web.api.dto.ControllerDTO;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+
+public class NiFiRestApiUtil {
+    public static final int RESPONSE_CODE_OK = 200;
+    
+    private final SSLContext sslContext;
+    
+    public NiFiRestApiUtil(final SSLContext sslContext) {
+        this.sslContext = sslContext;
+    }
+    
+    private HttpURLConnection getConnection(final String connUrl, final int timeoutMillis) throws IOException {
+        final URL url = new URL(connUrl);
+        final HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+        connection.setConnectTimeout(timeoutMillis);
+        connection.setReadTimeout(timeoutMillis);
+        
+        // special handling for https
+        if (sslContext != null && connection instanceof HttpsURLConnection) {
+            HttpsURLConnection secureConnection = (HttpsURLConnection) connection;
+            secureConnection.setSSLSocketFactory(sslContext.getSocketFactory());
+
+            // check the trusted hostname property and override the HostnameVerifier
+            secureConnection.setHostnameVerifier(new OverrideHostnameVerifier(url.getHost(), 
+                    secureConnection.getHostnameVerifier()));
+        }
+        
+        return connection;
+    }
+    
+    public ControllerDTO getController(final String url, final int timeoutMillis) throws IOException {
+        final HttpURLConnection connection = getConnection(url, timeoutMillis);
+        connection.setRequestMethod("GET");
+        final int responseCode = connection.getResponseCode();
+        
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        StreamUtils.copy(connection.getInputStream(), baos);
+        final String responseMessage = baos.toString();
+        
+        if ( responseCode == RESPONSE_CODE_OK ) {
+            final ObjectMapper mapper = new ObjectMapper();
+            final JsonNode jsonNode = mapper.readTree(responseMessage);
+            final JsonNode controllerNode = jsonNode.get("controller");
+            return mapper.readValue(controllerNode, ControllerDTO.class);
+        } else {
+            throw new IOException("Got HTTP response Code " + responseCode + ": " + connection.getResponseMessage() + " with explanation: " + responseMessage);
+        }
+    }
+    
+    private static class OverrideHostnameVerifier implements HostnameVerifier {
+        private final String trustedHostname;
+        private final HostnameVerifier delegate;
+
+        private OverrideHostnameVerifier(String trustedHostname, HostnameVerifier delegate) {
+            this.trustedHostname = trustedHostname;
+            this.delegate = delegate;
+        }
+
+        @Override
+        public boolean verify(String hostname, SSLSession session) {
+            if (trustedHostname.equalsIgnoreCase(hostname)) {
+                return true;
+            }
+            return delegate.verify(hostname, session);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java
deleted file mode 100644
index b2dbdcd..0000000
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java
+++ /dev/null
@@ -1,216 +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.util;
-
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.Map;
-
-import javax.net.ssl.SSLContext;
-import javax.ws.rs.core.MediaType;
-
-import org.apache.nifi.web.api.dto.ControllerDTO;
-import org.apache.nifi.web.api.entity.ControllerEntity;
-import org.apache.nifi.web.util.WebUtils;
-
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientHandlerException;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.ClientResponse.Status;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.UniformInterfaceException;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
-
-/**
- *
- */
-public class RemoteNiFiUtils {
-
-    public static final String CONTROLLER_URI_PATH = "/controller";
-
-    private static final int CONNECT_TIMEOUT = 10000;
-    private static final int READ_TIMEOUT = 10000;
-    
-    private final Client client;
-    
-    public RemoteNiFiUtils(final SSLContext sslContext) {
-        this.client = getClient(sslContext);
-    }
-    
-
-    /**
-     * Gets the content at the specified URI.
-     *
-     * @param uri
-     * @param timeoutMillis
-     * @return
-     * @throws ClientHandlerException
-     * @throws UniformInterfaceException
-     */
-    public ClientResponse get(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
-        return get(uri, timeoutMillis, null);
-    }
-    
-    /**
-     * Gets the content at the specified URI using the given query parameters.
-     *
-     * @param uri
-     * @param timeoutMillis
-     * @param queryParams 
-     * @return
-     * @throws ClientHandlerException
-     * @throws UniformInterfaceException
-     */
-    public ClientResponse get(final URI uri, final int timeoutMillis, final Map<String, String> queryParams) throws ClientHandlerException, UniformInterfaceException {
-        // perform the request
-        WebResource webResource = client.resource(uri);
-        if ( queryParams != null ) {
-            for ( final Map.Entry<String, String> queryEntry : queryParams.entrySet() ) {
-                webResource = webResource.queryParam(queryEntry.getKey(), queryEntry.getValue());
-            }
-        }
-
-        webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
-        webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
-
-        return webResource.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    }
-
-    /**
-     * Performs a HEAD request to the specified URI.
-     *
-     * @param uri
-     * @param timeoutMillis
-     * @return
-     * @throws ClientHandlerException
-     * @throws UniformInterfaceException
-     */
-    public ClientResponse head(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
-        // perform the request
-        WebResource webResource = client.resource(uri);
-        webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
-        webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
-        return webResource.head();
-    }
-
-    /**
-     * Gets a client based on the specified URI.
-     * 
-     * @param uri
-     * @return 
-     */
-    private Client getClient(final SSLContext sslContext) {
-        final Client client;
-        if (sslContext == null) {
-            client = WebUtils.createClient(null);
-        } else {
-            client = WebUtils.createClient(null, sslContext);
-        }
-
-        client.setReadTimeout(READ_TIMEOUT);
-        client.setConnectTimeout(CONNECT_TIMEOUT);
-
-        return client;
-    }
-    
-    
-    /**
-     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
-     * is not configured to use Site-to-Site transfers.
-     * 
-     * @param uri the base URI of the remote instance. This should include the path only to the nifi-api level, as well as the protocol, host, and port.
-     * @param timeoutMillis
-     * @return
-     * @throws IOException
-     */
-    public Integer getRemoteListeningPort(final String uri, final int timeoutMillis) throws IOException {
-    	try {
-			final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
-			return getRemoteListeningPort(uriObject, timeoutMillis);
-		} catch (URISyntaxException e) {
-			throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
-		}
-    }
-    
-    public String getRemoteRootGroupId(final String uri, final int timeoutMillis) throws IOException {
-        try {
-            final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
-            return getRemoteRootGroupId(uriObject, timeoutMillis);
-        } catch (URISyntaxException e) {
-            throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
-        }
-    }
-    
-    public String getRemoteInstanceId(final String uri, final int timeoutMillis) throws IOException {
-        try {
-            final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
-            return getController(uriObject, timeoutMillis).getInstanceId();
-        } catch (URISyntaxException e) {
-            throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
-        }
-    }
-    
-    /**
-     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
-     * is not configured to use Site-to-Site transfers.
-     * 
-     * @param uri the full URI to fetch, including the path.
-     * @return
-     * @throws IOException
-     */
-    private Integer getRemoteListeningPort(final URI uri, final int timeoutMillis) throws IOException {
-    	return getController(uri, timeoutMillis).getRemoteSiteListeningPort();
-    }
-    
-    private String getRemoteRootGroupId(final URI uri, final int timeoutMillis) throws IOException {
-        return getController(uri, timeoutMillis).getId();
-    }
-    
-    public ControllerDTO getController(final URI uri, final int timeoutMillis) throws IOException {
-        final ClientResponse response = get(uri, timeoutMillis);
-        
-        if (Status.OK.getStatusCode() == response.getStatusInfo().getStatusCode()) {
-            final ControllerEntity entity = response.getEntity(ControllerEntity.class);
-            return entity.getController();
-        } else {
-            final String responseMessage = response.getEntity(String.class);
-            throw new IOException("Got HTTP response Code " + response.getStatusInfo().getStatusCode() + ": " + response.getStatusInfo().getReasonPhrase() + " with explanation: " + responseMessage);
-        }
-    }
-    
-    /**
-     * Issues a registration request on behalf of the current user.
-     * 
-     * @param baseApiUri 
-     * @return  
-     */
-    public ClientResponse issueRegistrationRequest(String baseApiUri) {
-        final URI uri = URI.create(String.format("%s/%s", baseApiUri, "/controller/users"));
-
-        // set up the query params
-        MultivaluedMapImpl entity = new MultivaluedMapImpl();
-        entity.add("justification", "A Remote instance of NiFi has attempted to create a reference to this NiFi. This action must be approved first.");
-        
-        // create the web resource
-        WebResource webResource = client.resource(uri);
-        
-        // get the client utils and make the request
-        return webResource.accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_FORM_URLENCODED).entity(entity).post(ClientResponse.class);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
index 8781421..bb16a34 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
@@ -36,7 +36,7 @@ import org.junit.Test;
 public class TestSiteToSiteClient {
 
     @Test
-    @Ignore("For local testing only; not really a unit test but a manual test")
+    //@Ignore("For local testing only; not really a unit test but a manual test")
     public void testReceive() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         
@@ -58,7 +58,6 @@ public class TestSiteToSiteClient {
             final byte[] buff = new byte[(int) size];
             
             StreamUtils.fillBuffer(in, buff);
-            System.out.println(buff.length);
             
             Assert.assertNull(transaction.receive());
             
@@ -71,7 +70,7 @@ public class TestSiteToSiteClient {
     
     
     @Test
-    @Ignore("For local testing only; not really a unit test but a manual test")
+    //@Ignore("For local testing only; not really a unit test but a manual test")
     public void testSend() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         
@@ -84,12 +83,12 @@ public class TestSiteToSiteClient {
             final Transaction transaction = client.createTransaction(TransferDirection.SEND);
             Assert.assertNotNull(transaction);
             
-                final Map<String, String> attrs = new HashMap<>();
-                attrs.put("site-to-site", "yes, please!");
-                final byte[] bytes = "Hello".getBytes();
-                final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-                final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
-                transaction.send(packet);
+            final Map<String, String> attrs = new HashMap<>();
+            attrs.put("site-to-site", "yes, please!");
+            final byte[] bytes = "Hello".getBytes();
+            final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+            final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
+            transaction.send(packet);
             
             transaction.confirm();
             transaction.complete();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java
new file mode 100644
index 0000000..23dfdda
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.net.URISyntaxException;
+import java.util.Map;
+
+import javax.net.ssl.SSLContext;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.nifi.web.api.dto.ControllerDTO;
+import org.apache.nifi.web.api.entity.ControllerEntity;
+import org.apache.nifi.web.util.WebUtils;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.ClientResponse.Status;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+
+/**
+ *
+ */
+public class RemoteNiFiUtils {
+
+    public static final String CONTROLLER_URI_PATH = "/controller";
+
+    private static final int CONNECT_TIMEOUT = 10000;
+    private static final int READ_TIMEOUT = 10000;
+    
+    private final Client client;
+    
+    public RemoteNiFiUtils(final SSLContext sslContext) {
+        this.client = getClient(sslContext);
+    }
+    
+
+    /**
+     * Gets the content at the specified URI.
+     *
+     * @param uri
+     * @param timeoutMillis
+     * @return
+     * @throws ClientHandlerException
+     * @throws UniformInterfaceException
+     */
+    public ClientResponse get(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
+        return get(uri, timeoutMillis, null);
+    }
+    
+    /**
+     * Gets the content at the specified URI using the given query parameters.
+     *
+     * @param uri
+     * @param timeoutMillis
+     * @param queryParams 
+     * @return
+     * @throws ClientHandlerException
+     * @throws UniformInterfaceException
+     */
+    public ClientResponse get(final URI uri, final int timeoutMillis, final Map<String, String> queryParams) throws ClientHandlerException, UniformInterfaceException {
+        // perform the request
+        WebResource webResource = client.resource(uri);
+        if ( queryParams != null ) {
+            for ( final Map.Entry<String, String> queryEntry : queryParams.entrySet() ) {
+                webResource = webResource.queryParam(queryEntry.getKey(), queryEntry.getValue());
+            }
+        }
+
+        webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
+        webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
+
+        return webResource.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    }
+
+    /**
+     * Performs a HEAD request to the specified URI.
+     *
+     * @param uri
+     * @param timeoutMillis
+     * @return
+     * @throws ClientHandlerException
+     * @throws UniformInterfaceException
+     */
+    public ClientResponse head(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
+        // perform the request
+        WebResource webResource = client.resource(uri);
+        webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
+        webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
+        return webResource.head();
+    }
+
+    /**
+     * Gets a client based on the specified URI.
+     * 
+     * @param uri
+     * @return 
+     */
+    private Client getClient(final SSLContext sslContext) {
+        final Client client;
+        if (sslContext == null) {
+            client = WebUtils.createClient(null);
+        } else {
+            client = WebUtils.createClient(null, sslContext);
+        }
+
+        client.setReadTimeout(READ_TIMEOUT);
+        client.setConnectTimeout(CONNECT_TIMEOUT);
+
+        return client;
+    }
+    
+    
+    /**
+     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
+     * is not configured to use Site-to-Site transfers.
+     * 
+     * @param uri the base URI of the remote instance. This should include the path only to the nifi-api level, as well as the protocol, host, and port.
+     * @param timeoutMillis
+     * @return
+     * @throws IOException
+     */
+    public Integer getRemoteListeningPort(final String uri, final int timeoutMillis) throws IOException {
+    	try {
+			final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
+			return getRemoteListeningPort(uriObject, timeoutMillis);
+		} catch (URISyntaxException e) {
+			throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
+		}
+    }
+    
+    public String getRemoteRootGroupId(final String uri, final int timeoutMillis) throws IOException {
+        try {
+            final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
+            return getRemoteRootGroupId(uriObject, timeoutMillis);
+        } catch (URISyntaxException e) {
+            throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
+        }
+    }
+    
+    public String getRemoteInstanceId(final String uri, final int timeoutMillis) throws IOException {
+        try {
+            final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
+            return getController(uriObject, timeoutMillis).getInstanceId();
+        } catch (URISyntaxException e) {
+            throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
+        }
+    }
+    
+    /**
+     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
+     * is not configured to use Site-to-Site transfers.
+     * 
+     * @param uri the full URI to fetch, including the path.
+     * @return
+     * @throws IOException
+     */
+    private Integer getRemoteListeningPort(final URI uri, final int timeoutMillis) throws IOException {
+    	return getController(uri, timeoutMillis).getRemoteSiteListeningPort();
+    }
+    
+    private String getRemoteRootGroupId(final URI uri, final int timeoutMillis) throws IOException {
+        return getController(uri, timeoutMillis).getId();
+    }
+    
+    public ControllerDTO getController(final URI uri, final int timeoutMillis) throws IOException {
+        final ClientResponse response = get(uri, timeoutMillis);
+        
+        if (Status.OK.getStatusCode() == response.getStatusInfo().getStatusCode()) {
+            final ControllerEntity entity = response.getEntity(ControllerEntity.class);
+            return entity.getController();
+        } else {
+            final String responseMessage = response.getEntity(String.class);
+            throw new IOException("Got HTTP response Code " + response.getStatusInfo().getStatusCode() + ": " + response.getStatusInfo().getReasonPhrase() + " with explanation: " + responseMessage);
+        }
+    }
+    
+    /**
+     * Issues a registration request on behalf of the current user.
+     * 
+     * @param baseApiUri 
+     * @return  
+     */
+    public ClientResponse issueRegistrationRequest(String baseApiUri) {
+        final URI uri = URI.create(String.format("%s/%s", baseApiUri, "/controller/users"));
+
+        // set up the query params
+        MultivaluedMapImpl entity = new MultivaluedMapImpl();
+        entity.add("justification", "A Remote instance of NiFi has attempted to create a reference to this NiFi. This action must be approved first.");
+        
+        // create the web resource
+        WebResource webResource = client.resource(uri);
+        
+        // get the client utils and make the request
+        return webResource.accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_FORM_URLENCODED).entity(entity).post(ClientResponse.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e16fc797/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 79ef7a8..6b70fe6 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -56,7 +56,6 @@ 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.remote.util.RemoteNiFiUtils;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.util.FormatUtils;


[09/29] incubator-nifi git commit: NIFI-282: Refactoring to allow for separate client

Posted by ma...@apache.org.
NIFI-282: Refactoring to allow for separate 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/081471c4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/081471c4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/081471c4

Branch: refs/heads/develop
Commit: 081471c420f113f0eb1440df74f7dff0e04067ec
Parents: 05b6459
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 9 20:41:39 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 9 20:41:39 2015 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/remote/Peer.java  |  31 +-
 .../org/apache/nifi/remote/Transaction.java     |  10 +
 .../nifi/remote/client/SiteToSiteClient.java    |  14 +-
 .../client/socket/EndpointConnection.java       |  54 ++
 .../client/socket/EndpointConnectionPool.java   | 855 +++++++++++++++++++
 .../client/socket/EndpointConnectionState.java  |  54 --
 .../socket/EndpointConnectionStatePool.java     | 835 ------------------
 .../nifi/remote/client/socket/SocketClient.java |  86 +-
 .../remote/exception/HandshakeException.java    |   9 +-
 .../exception/PortNotRunningException.java      |   8 +-
 .../remote/exception/ProtocolException.java     |   4 +
 .../remote/exception/UnknownPortException.java  |   7 +-
 .../remote/io/socket/SocketChannelInput.java    |   5 +
 .../io/socket/ssl/SSLSocketChannelInput.java    |   5 +
 .../nifi/remote/protocol/ClientProtocol.java    |   4 +-
 .../remote/protocol/CommunicationsInput.java    |   6 +
 .../protocol/socket/SocketClientProtocol.java   |  31 +-
 .../socket/SocketClientTransaction.java         |  19 +-
 .../socket/TestEndpointConnectionStatePool.java |   8 +-
 .../client/socket/TestSiteToSiteClient.java     | 100 +++
 .../io/socket/SocketChannelInputStream.java     |  10 +
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  10 +
 .../socket/ssl/SSLSocketChannelInputStream.java |   4 +
 .../apache/nifi/groups/RemoteProcessGroup.java  |   4 -
 .../apache/nifi/controller/FlowController.java  |   5 +-
 .../nifi/remote/StandardRemoteProcessGroup.java |  14 +-
 .../nifi/remote/StandardRemoteGroupPort.java    |  63 +-
 .../SSLSocketChannelCommunicationsSession.java  |  93 --
 .../io/socket/ssl/SSLSocketChannelInput.java    |  50 --
 .../io/socket/ssl/SSLSocketChannelOutput.java   |  44 -
 .../socket/SocketFlowFileServerProtocol.java    |   2 +-
 31 files changed, 1250 insertions(+), 1194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
index 29af777..dda5ae3 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
@@ -18,10 +18,10 @@ package org.apache.nifi.remote;
 
 import java.io.IOException;
 import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.nifi.remote.protocol.CommunicationsSession;
-import org.apache.nifi.stream.io.NullOutputStream;
-import org.apache.nifi.stream.io.StreamUtils;
 
 public class Peer {
 
@@ -29,7 +29,8 @@ public class Peer {
     private final String url;
     private final String clusterUrl;
     private final String host;
-    private long penalizationExpiration = 0L;
+    
+    private final Map<String, Long> penaltyExpirationMap = new HashMap<>();
     private boolean closed = false;
 
     public Peer(final CommunicationsSession commsSession, final String peerUrl, final String clusterUrl) {
@@ -61,19 +62,31 @@ public class Peer {
 
         // Consume the InputStream so that it doesn't linger on the Peer's outgoing socket buffer
         try {
-            StreamUtils.copy(commsSession.getInput().getInputStream(), new NullOutputStream());
+            commsSession.getInput().consume();
         } finally {
             commsSession.close();
         }
     }
 
-    public void penalize(final long millis) {
-        penalizationExpiration = Math.max(penalizationExpiration, System.currentTimeMillis() + millis);
+    /**
+     * Penalizes this peer for the given destination only for the provided number of milliseconds
+     * @param destinationId
+     * @param millis
+     */
+    public void penalize(final String destinationId, final long millis) {
+        final Long currentPenalty = penaltyExpirationMap.get(destinationId);
+        final long proposedPenalty = System.currentTimeMillis() + millis;
+        if ( currentPenalty == null || proposedPenalty > currentPenalty ) {
+            penaltyExpirationMap.put(destinationId, proposedPenalty);
+        }
     }
+    
 
-    public boolean isPenalized() {
-        return penalizationExpiration > System.currentTimeMillis();
+    public boolean isPenalized(final String destinationId) {
+        final Long currentPenalty = penaltyExpirationMap.get(destinationId);
+        return (currentPenalty != null && currentPenalty > System.currentTimeMillis());
     }
+    
 
     public boolean isClosed() {
         return closed;
@@ -110,8 +123,6 @@ public class Peer {
         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/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
index cc16625..9fb6147 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
@@ -137,6 +137,16 @@ public interface Transaction {
 	void complete(boolean requestBackoff) throws IOException;
 	
 	/**
+     * <p>
+     * Completes the transaction and indicates to both the sender and receiver that the data transfer was
+     * successful.
+     * </p>
+     * 
+     * @throws IOException
+     */
+	void complete() throws IOException;
+	
+	/**
 	 * <p>
 	 * Cancels this transaction, indicating to the sender that the data has not been successfully received so that
 	 * the sender can retry or handle however is appropriate.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
index fa94b81..47568fd 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -27,6 +27,10 @@ import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.socket.SocketClient;
+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;
 import org.apache.nifi.remote.protocol.DataPacket;
 
 /**
@@ -65,18 +69,24 @@ import org.apache.nifi.remote.protocol.DataPacket;
 public interface SiteToSiteClient extends Closeable {
 
 	/**
+	 * <p>
 	 * Creates a new Transaction that can be used to either send data to a remote NiFi instance
 	 * or receive data from a remote NiFi instance, depending on the value passed for the {@code direction} argument.
+	 * </p>
 	 * 
+	 * <p>
+	 * <b>Note:</b> If all of the nodes are penalized (See {@link Builder#nodePenalizationPeriod(long, TimeUnit)}), then
+	 * this method will return <code>null</code>.
+	 * </p>
 	 * 
 	 * @param direction specifies which direction the data should be transferred. A value of {@link TransferDirection#SEND}
 	 * indicates that this Transaction will send data to the remote instance; a value of {@link TransferDirection#RECEIVE} indicates
 	 * that this Transaction will be used to receive data from the remote instance.
 	 * 
-	 * @return
+	 * @return a Transaction to use for sending or receiving data, or <code>null</code> if all nodes are penalized.
 	 * @throws IOException
 	 */
-	Transaction createTransaction(TransferDirection direction) throws IOException;
+	Transaction createTransaction(TransferDirection direction) throws HandshakeException, PortNotRunningException, ProtocolException, UnknownPortException, IOException;
 	
 	/**
 	 * <p>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnection.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnection.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnection.java
new file mode 100644
index 0000000..651ae50
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnection.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 EndpointConnection {
+	private final Peer peer;
+    private final SocketClientProtocol socketClientProtocol;
+    private final FlowFileCodec codec;
+    private volatile long lastUsed;
+    
+    public EndpointConnection(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/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
new file mode 100644
index 0000000..6869cca
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -0,0 +1,855 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
+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 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.client.SiteToSiteClientConfig;
+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.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.remote.util.RemoteNiFiUtils;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.web.api.dto.ControllerDTO;
+import org.apache.nifi.web.api.dto.PortDTO;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class EndpointConnectionPool {
+    public static final long PEER_REFRESH_PERIOD = 60000L;
+    public static final String CATEGORY = "Site-to-Site";
+    public static final long REMOTE_REFRESH_MILLIS = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
+
+    private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
+
+	private static final Logger logger = LoggerFactory.getLogger(EndpointConnectionPool.class);
+	
+	private final BlockingQueue<EndpointConnection> connectionQueue = new LinkedBlockingQueue<>();
+    private final ConcurrentMap<PeerStatus, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
+    private final URI clusterUrl;
+    private final String apiUri;
+    
+    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;
+    private final ScheduledExecutorService taskExecutor;
+    
+    private final ReadWriteLock listeningPortRWLock = new ReentrantReadWriteLock();
+    private final Lock remoteInfoReadLock = listeningPortRWLock.readLock();
+    private final Lock remoteInfoWriteLock = listeningPortRWLock.writeLock();
+    private Integer siteToSitePort;
+    private Boolean siteToSiteSecure;
+    private long remoteRefreshTime;
+    private final Map<String, String> inputPortMap = new HashMap<>();	// map input port name to identifier
+    private final Map<String, String> outputPortMap = new HashMap<>();	// map output port name to identifier
+    
+    private volatile int commsTimeout;
+
+    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final EventReporter eventReporter, final File persistenceFile) {
+    	this(clusterUrl, commsTimeoutMillis, null, eventReporter, persistenceFile);
+    }
+    
+    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) {
+    	try {
+    		this.clusterUrl = new URI(clusterUrl);
+    	} catch (final URISyntaxException e) {
+    		throw new IllegalArgumentException("Invalid Cluster URL: " + clusterUrl);
+    	}
+    	
+    	// Trim the trailing /
+        String uriPath = this.clusterUrl.getPath();
+        if (uriPath.endsWith("/")) {
+            uriPath = uriPath.substring(0, uriPath.length() - 1);
+        }
+        apiUri = this.clusterUrl.getScheme() + "://" + this.clusterUrl.getHost() + ":" + this.clusterUrl.getPort() + uriPath + "-api";
+        
+    	this.sslContext = sslContext;
+    	this.peersFile = persistenceFile;
+    	this.eventReporter = eventReporter;
+    	this.commsTimeout = commsTimeoutMillis;
+    	
+    	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;
+    	}
+
+    	// Initialize a scheduled executor and run some maintenance tasks in the background to kill off old, unused
+    	// connections and keep our list of peers up-to-date.
+    	taskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
+    		private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+    		
+			@Override
+			public Thread newThread(final Runnable r) {
+				final Thread thread = defaultFactory.newThread(r);
+				thread.setName("NiFi Site-to-Site Connection Pool Maintenance");
+				return thread;
+			}
+    	});
+
+    	taskExecutor.scheduleWithFixedDelay(new Runnable() {
+			@Override
+			public void run() {
+				refreshPeers();
+			}
+    	}, 0, 5, TimeUnit.SECONDS);
+
+    	taskExecutor.scheduleWithFixedDelay(new Runnable() {
+			@Override
+			public void run() {
+				cleanupExpiredSockets();
+			}
+    	}, 5, 5, TimeUnit.SECONDS);
+    }
+    
+    
+    public EndpointConnection getEndpointConnection(final RemoteDestination remoteDestination, final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
+        return getEndpointConnection(remoteDestination, direction, null);
+    }
+    
+    
+    public EndpointConnection getEndpointConnection(final RemoteDestination remoteDestination, final TransferDirection direction, final SiteToSiteClientConfig config) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
+    	//
+        // Attempt to get a connection state that already exists for this URL.
+        //
+        FlowFileCodec codec = null;
+        CommunicationsSession commsSession = null;
+        SocketClientProtocol protocol = null;
+        EndpointConnection connection;
+        Peer peer = null;
+        
+        final List<EndpointConnection> addBack = new ArrayList<>();
+        try {
+            do {
+                connection = connectionQueue.poll();
+                logger.debug("{} Connection State for {} = {}", this, clusterUrl, connection);
+                
+                if ( connection == null && !addBack.isEmpty() ) {
+                    // all available connections have been penalized.
+                    logger.debug("{} all Connections for {} are penalized; returning no Connection", this, remoteDestination.getIdentifier());
+                    return null;
+                }
+                
+                if ( connection != null && connection.getPeer().isPenalized(remoteDestination.getIdentifier()) ) {
+                    // we have a connection, but it's penalized. We want to add it back to the queue
+                    // when we've found one to use.
+                    addBack.add(connection);
+                    continue;
+                }
+                
+                // if we can't get an existing Connection, create one
+                if ( connection == null ) {
+                    logger.debug("No Connection available for Port {}; creating new Connection", remoteDestination.getIdentifier());
+                    protocol = new SocketClientProtocol();
+                    protocol.setDestination(remoteDestination);
+
+                    final PeerStatus peerStatus = getNextPeerStatus(direction);
+                    if ( peerStatus == null ) {
+                        return null;
+                    }
+
+                    try {
+                        commsSession = establishSiteToSiteConnection(peerStatus);
+                    } catch (final IOException ioe) {
+                        // TODO: penalize peer status
+                        penalize(peerStatus, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
+                        throw ioe;
+                    }
+                    
+                    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;
+                        }
+                    }
+                
+                    final String peerUrl = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort();
+                    peer = new Peer(commsSession, peerUrl, clusterUrl.toString());
+    
+                    // set properties based on config
+                    if ( config != null ) {
+                        protocol.setTimeout((int) config.getTimeout(TimeUnit.MILLISECONDS));
+                        protocol.setPreferredBatchCount(config.getPreferredBatchCount());
+                        protocol.setPreferredBatchSize(config.getPreferredBatchSize());
+                        protocol.setPreferredBatchDuration(config.getPreferredBatchDuration(TimeUnit.MILLISECONDS));
+                    }
+                    
+                    // 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));
+                            connectionQueue.offer(connection);
+                            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;
+                    }
+                    
+                    connection = new EndpointConnection(peer, protocol, codec);
+                } else {
+                    final long lastTimeUsed = connection.getLastTimeUsed();
+                    final long millisSinceLastUse = System.currentTimeMillis() - lastTimeUsed;
+                    
+                    if ( commsTimeout > 0L && millisSinceLastUse >= commsTimeout ) {
+                        cleanup(connection.getSocketClientProtocol(), connection.getPeer());
+                        connection = null;
+                    } else {
+                        codec = connection.getCodec();
+                        peer = connection.getPeer();
+                        commsSession = peer.getCommunicationsSession();
+                        protocol = connection.getSocketClientProtocol();
+                    }
+                }
+            } while ( connection == null || codec == null || commsSession == null || protocol == null );
+        } finally {
+            if ( !addBack.isEmpty() ) {
+                connectionQueue.addAll(addBack);
+            }
+        }
+        
+        return connection;
+    }
+    
+    
+    public boolean offer(final EndpointConnection endpointConnection) {
+    	final Peer peer = endpointConnection.getPeer();
+    	if ( peer == null ) {
+    		return false;
+    	}
+    	
+    	final String url = peer.getUrl();
+    	if ( url == null ) {
+    		return false;
+    	}
+    	
+    	return connectionQueue.offer(endpointConnection);
+    }
+    
+    private void penalize(final PeerStatus status, final long penalizationMillis) {
+        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));
+    }
+    
+    /**
+     * 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);
+        penalize(status, penalizationMillis);
+    }
+    
+    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.lock();
+            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);
+                    }
+                    
+                    if ( eventReporter != null ) {
+                    	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, HandshakeException, UnknownPortException, PortNotRunningException {
+        Set<PeerStatus> statuses = getPeerStatuses();
+        if ( statuses == null ) {
+            refreshPeers();
+            statuses = getPeerStatuses();
+            if ( statuses == null ) {
+                logger.debug("{} found no peers to connect to", this);
+                return Collections.emptyList();
+            }
+        }
+        
+        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);
+    }
+    
+    
+    private 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() throws IOException, HandshakeException, UnknownPortException, PortNotRunningException {
+    	final String hostname = clusterUrl.getHost();
+        final int port = getSiteToSitePort();
+    	
+    	final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port);
+        final Peer peer = new Peer(commsSession, "nifi://" + hostname + ":" + port, clusterUrl.toString());
+        final SocketClientProtocol clientProtocol = new SocketClientProtocol();
+        final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+        final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+        RemoteResourceInitiator.initiateResourceNegotiation(clientProtocol, dis, dos);
+
+        clientProtocol.setTimeout(commsTimeout);
+        clientProtocol.handshake(peer, null);
+        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;
+    }
+    
+    
+    private CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException {
+    	return establishSiteToSiteConnection(peerStatus.getHostname(), peerStatus.getPort());
+    }
+    
+    private CommunicationsSession establishSiteToSiteConnection(final String hostname, final int port) throws IOException {
+    	final boolean siteToSiteSecure = isSecure();
+        final String destinationUri = "nifi://" + hostname + ":" + port;
+
+        CommunicationsSession commsSession = null;
+        try {
+	        if ( siteToSiteSecure ) {
+	            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;
+    }
+    
+    
+    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;
+    }
+    
+    
+    private void cleanupExpiredSockets() {
+        final List<EndpointConnection> states = new ArrayList<>();
+        
+        EndpointConnection state;
+        while ((state = connectionQueue.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);
+            }
+        }
+        
+        connectionQueue.addAll(states);
+    }
+    
+    public void shutdown() {
+    	taskExecutor.shutdown();
+    	peerTimeoutExpirations.clear();
+            
+        for ( final CommunicationsSession commsSession : activeCommsChannels ) {
+            commsSession.interrupt();
+        }
+        
+        EndpointConnection state;
+        while ( (state = connectionQueue.poll()) != null)  {
+            cleanup(state.getSocketClientProtocol(), state.getPeer());
+        }
+    }
+    
+    public void terminate(final EndpointConnection state) {
+        cleanup(state.getSocketClientProtocol(), state.getPeer());
+    }
+    
+    private void refreshPeers() {
+        final PeerStatusCache existingCache = peerStatusCache;
+        if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
+            return;
+        }
+
+        try {
+            final Set<PeerStatus> statuses = fetchRemotePeerStatuses();
+            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);
+            }
+        }
+    }
+    
+    
+    public String getInputPortIdentifier(final String portName) throws IOException {
+        return getPortIdentifier(portName, inputPortMap);
+    }
+    
+    public String getOutputPortIdentifier(final String portName) throws IOException {
+    	return getPortIdentifier(portName, outputPortMap);
+    }
+    
+    
+    private String getPortIdentifier(final String portName, final Map<String, String> portMap) throws IOException {
+    	String identifier;
+    	remoteInfoReadLock.lock();
+        try {
+        	identifier = portMap.get(portName);
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+        
+        if ( identifier != null ) {
+        	return identifier;
+        }
+        
+        refreshRemoteInfo();
+
+    	remoteInfoReadLock.lock();
+        try {
+        	return portMap.get(portName);
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+    }
+    
+    
+    private ControllerDTO refreshRemoteInfo() throws IOException {
+    	final boolean webInterfaceSecure = clusterUrl.toString().startsWith("https");
+        final RemoteNiFiUtils utils = new RemoteNiFiUtils(webInterfaceSecure ? sslContext : null);
+		final ControllerDTO controller = utils.getController(URI.create(apiUri + "/controller"), commsTimeout);
+        
+        remoteInfoWriteLock.lock();
+        try {
+            this.siteToSitePort = controller.getRemoteSiteListeningPort();
+            this.siteToSiteSecure = controller.isSiteToSiteSecure();
+            
+            inputPortMap.clear();
+            for (final PortDTO inputPort : controller.getInputPorts()) {
+            	inputPortMap.put(inputPort.getName(), inputPort.getId());
+            }
+            
+            outputPortMap.clear();
+            for ( final PortDTO outputPort : controller.getOutputPorts()) {
+            	outputPortMap.put(outputPort.getName(), outputPort.getId());
+            }
+            
+            this.remoteRefreshTime = System.currentTimeMillis();
+        } finally {
+        	remoteInfoWriteLock.unlock();
+        }
+        
+        return controller;
+    }
+    
+    /**
+     * @return the port that the remote instance is listening on for
+     * site-to-site communication, or <code>null</code> if the remote instance
+     * is not configured to allow site-to-site communications.
+     *
+     * @throws IOException if unable to communicate with the remote instance
+     */
+    private Integer getSiteToSitePort() throws IOException {
+        Integer listeningPort;
+        remoteInfoReadLock.lock();
+        try {
+            listeningPort = this.siteToSitePort;
+            if (listeningPort != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
+                return listeningPort;
+            }
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+
+        final ControllerDTO controller = refreshRemoteInfo();
+        listeningPort = controller.getRemoteSiteListeningPort();
+
+        return listeningPort;
+    }
+ 
+    
+    
+    
+    /**
+     * Returns {@code true} if the remote instance is configured for secure site-to-site communications,
+     * {@code false} otherwise.
+     * 
+     * @return
+     * @throws IOException
+     */
+    public boolean isSecure() throws IOException {
+        remoteInfoReadLock.lock();
+        try {
+            final Boolean secure = this.siteToSiteSecure;
+            if (secure != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
+                return secure;
+            }
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+
+        final ControllerDTO controller = refreshRemoteInfo();
+        final Boolean isSecure = controller.isSiteToSiteSecure();
+        if ( isSecure == null ) {
+            throw new IOException("Remote NiFi instance " + clusterUrl + " is not currently configured to accept site-to-site connections");
+        }
+        
+        return isSecure;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.java
deleted file mode 100644
index f4ac727..0000000
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionState.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.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;
-    }
-}


[04/29] incubator-nifi git commit: NIFI-282: Bug fixes; documentation improvements; removed code marked as 'FOR TESTING'

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
new file mode 100644
index 0000000..befbdaa
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/ClientProtocol.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+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();
+
+    
+    
+    
+    Transaction startTransaction(Peer peer, FlowFileCodec codec, TransferDirection direction) throws IOException;
+    
+    
+    /**
+     * 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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsInput.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsOutput.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/CommunicationsSession.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java
new file mode 100644
index 0000000..f4fa4d0
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.protocol;
+
+import java.io.InputStream;
+import java.util.Map;
+
+public interface DataPacket {
+
+	Map<String, String> getAttributes();
+	
+	InputStream getData();
+	
+	long getSize();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/RequestType.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
new file mode 100644
index 0000000..41dc276
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+
+/**
+ * Enumeration of Properties that can be used for the Site-to-Site Socket Protocol.
+ */
+public enum HandshakeProperty {
+    /**
+     * Boolean value indicating whether or not the contents of a FlowFile should be
+     * GZipped when transferred.
+     */
+    GZIP,
+    
+    /**
+     * The unique identifier of the port to communicate with
+     */
+    PORT_IDENTIFIER,
+    
+    /**
+     * Indicates the number of milliseconds after the request was made that the client
+     * will wait for a response. If no response has been received by the time this value
+     * expires, the server can move on without attempting to service the request because
+     * the client will have already disconnected.
+     */
+    REQUEST_EXPIRATION_MILLIS,
+    
+    /**
+     * The preferred number of FlowFiles that the server should send to the client
+     * when pulling data. This property was introduced in version 5 of the protocol.
+     */
+    BATCH_COUNT,
+    
+    /**
+     * The preferred number of bytes that the server should send to the client when
+     * pulling data. This property was introduced in version 5 of the protocol.
+     */
+    BATCH_SIZE,
+    
+    /**
+     * The preferred amount of time that the server should send data to the client
+     * when pulling data. This property was introduced in version 5 of the protocol.
+     * Value is in milliseconds.
+     */
+    BATCH_DURATION;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/Response.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
new file mode 100644
index 0000000..8860e73
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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),
+    CANCEL_TRANSACTION(15, "Cancel Transaction", true),
+    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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
new file mode 100644
index 0000000..5f194f8
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+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.processor.io.InputStreamCallback;
+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.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+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.protocol.ClientProtocol;
+import org.apache.nifi.remote.protocol.CommunicationsSession;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.remote.protocol.RequestType;
+import org.apache.nifi.remote.util.StandardDataPacket;
+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(5, 4, 3, 2, 1);
+
+    private RemoteDestination destination;
+    private boolean useCompression = false;
+    
+    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 int timeoutMillis = 30000;
+    
+    private int batchCount;
+    private long batchSize;
+    private long batchMillis;
+
+    private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
+    
+    public SocketClientProtocol() {
+    }
+
+    public void setPreferredBatchCount(final int count) {
+        this.batchCount = count;
+    }
+    
+    public void setPreferredBatchSize(final long bytes) {
+        this.batchSize = bytes;
+    }
+    
+    public void setPreferredBatchDuration(final long millis) {
+        this.batchMillis = millis;
+    }
+    
+    public void setDestination(final RemoteDestination destination) {
+        this.destination = destination;
+        this.useCompression = destination.isUseCompression();
+    }
+    
+    public void setTimeout(final int timeoutMillis) {
+    	this.timeoutMillis = timeoutMillis;
+    }
+    
+    @Override
+    public void handshake(final Peer peer) throws IOException, HandshakeException {
+    	handshake(peer, destination.getIdentifier());
+    }
+    
+    public void handshake(final Peer peer, final String destinationId) 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) );
+        
+        if ( versionNegotiator.getVersion() >= 5 ) {
+            if ( batchCount > 0 ) {
+                properties.put(HandshakeProperty.BATCH_COUNT, String.valueOf(batchCount));
+            }
+            if ( batchSize > 0L ) {
+                properties.put(HandshakeProperty.BATCH_SIZE, String.valueOf(batchSize));
+            }
+            if ( batchMillis > 0L ) {
+                properties.put(HandshakeProperty.BATCH_DURATION, String.valueOf(batchMillis));
+            }
+        }
+        
+        final CommunicationsSession commsSession = peer.getCommunicationsSession();
+        commsSession.setTimeout(timeoutMillis);
+        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 Transaction startTransaction(final Peer peer, final FlowFileCodec codec, final TransferDirection direction) throws IOException, ProtocolException {
+        if ( !handshakeComplete ) {
+            throw new IllegalStateException("Handshake has not been performed");
+        }
+        if ( !readyForFileTransfer ) {
+            throw new IllegalStateException("Cannot start transaction; handshake resolution was " + handshakeResponse);
+        }
+        
+        return new SocketClientTransaction(versionNegotiator.getVersion(), peer, codec, 
+        		direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS));
+    }
+
+
+    @Override
+    public void receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+    	final String userDn = peer.getCommunicationsSession().getUserDn();
+    	final Transaction transaction = startTransaction(peer, codec, TransferDirection.RECEIVE);
+    	
+    	final StopWatch stopWatch = new StopWatch(true);
+    	final Set<FlowFile> flowFilesReceived = new HashSet<>();
+    	long bytesReceived = 0L;
+    	
+    	while (true) {
+    		final long start = System.nanoTime();
+    		final DataPacket dataPacket = transaction.receive();
+    		if ( dataPacket == null ) {
+    		    if ( flowFilesReceived.isEmpty() ) {
+    		        peer.penalize(destination.getYieldPeriod(TimeUnit.MILLISECONDS));
+    		    }
+    			break;
+    		}
+    		
+    		FlowFile flowFile = session.create();
+    		flowFile = session.putAllAttributes(flowFile, dataPacket.getAttributes());
+    		flowFile = session.importFrom(dataPacket.getData(), flowFile);
+    		final long receiveNanos = System.nanoTime() - start;
+    		
+			String sourceFlowFileIdentifier = dataPacket.getAttributes().get(CoreAttributes.UUID.key());
+			if ( sourceFlowFileIdentifier == null ) {
+				sourceFlowFileIdentifier = "<Unknown Identifier>";
+			}
+			
+			final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceFlowFileIdentifier;
+			session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, TimeUnit.NANOSECONDS.toMillis(receiveNanos));
+
+    		session.transfer(flowFile, Relationship.ANONYMOUS);
+    		bytesReceived += dataPacket.getSize();
+    	}
+
+    	// Confirm that what we received was the correct data.
+    	transaction.confirm();
+    	
+		// Commit the session so that we have persisted the data
+		session.commit();
+
+		// We want to apply backpressure if the outgoing connections are full. I.e., there are no available relationships.
+		final boolean applyBackpressure = context.getAvailableRelationships().isEmpty();
+
+		transaction.complete(applyBackpressure);
+		logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
+
+		if ( flowFilesReceived.isEmpty() ) {
+		    return;
+		}
+		
+		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 {
+		FlowFile flowFile = session.get();
+		if (flowFile == null) {
+			return;
+		}
+
+		try {
+			final String userDn = peer.getCommunicationsSession().getUserDn();
+			final long startSendingNanos = System.nanoTime();
+			final StopWatch stopWatch = new StopWatch(true);
+			long bytesSent = 0L;
+			
+			final Transaction transaction = startTransaction(peer, codec, TransferDirection.SEND);
+			
+			final Set<FlowFile> flowFilesSent = new HashSet<>();
+	        boolean continueTransaction = true;
+	        while (continueTransaction) {
+	        	final long startNanos = System.nanoTime();
+	            // call codec.encode within a session callback so that we have the InputStream to read the FlowFile
+	            final FlowFile toWrap = flowFile;
+	            session.read(flowFile, new InputStreamCallback() {
+	                @Override
+	                public void process(final InputStream in) throws IOException {
+	                    final DataPacket dataPacket = new StandardDataPacket(toWrap.getAttributes(), in, toWrap.getSize());
+	                    transaction.send(dataPacket);
+	                }
+	            });
+	            
+	            final long transferNanos = System.nanoTime() - startNanos;
+	            final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+	            
+	            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);
+	        }
+	        
+	        transaction.confirm();
+	        
+	        // 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();
+	        transaction.complete(false);
+	        
+	        final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
+	        logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
+	            this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+		} catch (final Exception e) {
+			session.rollback();
+			throw e;
+		}
+    }
+    
+    
+    @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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
new file mode 100644
index 0000000..edb360e
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
@@ -0,0 +1,357 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.OutputStream;
+import java.util.zip.CRC32;
+import java.util.zip.CheckedInputStream;
+import java.util.zip.CheckedOutputStream;
+
+import org.apache.nifi.remote.Peer;
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.codec.FlowFileCodec;
+import org.apache.nifi.remote.exception.ProtocolException;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.remote.protocol.RequestType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SocketClientTransaction implements Transaction {
+	private static final Logger logger = LoggerFactory.getLogger(SocketClientTransaction.class);
+	
+	
+	private final CRC32 crc = new CRC32();
+	private final int protocolVersion;
+	private final FlowFileCodec codec;
+	private final DataInputStream dis;
+	private final DataOutputStream dos;
+	private final TransferDirection direction;
+	private final boolean compress;
+	private final Peer peer;
+	private final int penaltyMillis;
+	
+	private boolean dataAvailable = false;
+	private int transfers = 0;
+	private TransactionState state;
+	
+	SocketClientTransaction(final int protocolVersion, final Peer peer, final FlowFileCodec codec, 
+			final TransferDirection direction, final boolean useCompression, final int penaltyMillis) throws IOException {
+		this.protocolVersion = protocolVersion;
+		this.peer = peer;
+		this.codec = codec;
+		this.direction = direction;
+		this.dis = new DataInputStream(peer.getCommunicationsSession().getInput().getInputStream());
+		this.dos = new DataOutputStream(peer.getCommunicationsSession().getOutput().getOutputStream());
+		this.compress = useCompression;
+		this.state = TransactionState.TRANSACTION_STARTED;
+		this.penaltyMillis = penaltyMillis;
+		
+		initialize();
+	}
+	
+	private void initialize() throws IOException {
+	    try {
+            if ( direction == TransferDirection.RECEIVE ) {
+                // Indicate that we would like to have some data
+                RequestType.RECEIVE_FLOWFILES.writeRequestType(dos);
+                dos.flush();
+                
+                final Response dataAvailableCode = Response.read(dis);
+                switch (dataAvailableCode.getCode()) {
+                    case MORE_DATA:
+                        logger.debug("{} {} Indicates that data is available", this, peer);
+                        this.dataAvailable = true;
+                        break;
+                    case NO_MORE_DATA:
+                        logger.debug("{} No data available from {}", peer);
+                        this.dataAvailable = false;
+                        return;
+                    default:
+                        throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
+                }
+    
+            } else {
+                // Indicate that we would like to have some data
+                RequestType.SEND_FLOWFILES.writeRequestType(dos);
+                dos.flush();
+            }
+	    } catch (final Exception e) {
+	        error();
+	        throw e;
+	    }
+	}
+	
+	
+	@Override
+	public DataPacket receive() throws IOException {
+	    try {
+    		if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
+    			throw new IllegalStateException("Cannot receive data because Transaction State is " + state);
+    		}
+    		
+        	if ( direction == TransferDirection.SEND ) {
+        	    throw new IllegalStateException("Attempting to receive data but started a SEND Transaction");
+        	}
+        	
+        	// if we already know there's no data, just return null
+        	if ( !dataAvailable ) {
+        	    return null;
+        	}
+    
+        	// if we have already received a packet, check if another is available.
+        	if ( transfers > 0 ) {
+        	    // Determine if Peer will send us data or has no data to send us
+                final Response dataAvailableCode = Response.read(dis);
+                switch (dataAvailableCode.getCode()) {
+                    case CONTINUE_TRANSACTION:
+                        logger.debug("{} {} Indicates Transaction should continue", this, peer);
+                        this.dataAvailable = true;
+                        break;
+                    case FINISH_TRANSACTION:
+                        logger.debug("{} {} Indicates Transaction should finish", peer);
+                        this.dataAvailable = false;
+                        break;
+                    default:
+                        throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
+                }
+            }
+        	
+        	// if no data available, return null
+        	if ( !dataAvailable ) {
+        	    return null;
+        	}
+        	
+            logger.debug("{} Receiving data from {}", this, peer);
+            final DataPacket packet = codec.decode(new CheckedInputStream(dis, crc));
+            
+            if ( packet == null ) {
+                this.dataAvailable = false;
+            } else {
+            	transfers++;
+            }
+            
+            this.state = TransactionState.DATA_EXCHANGED;
+            return packet;
+	    } catch (final Exception e) {
+	        error();
+	        throw e;
+	    }
+	}
+	
+	
+	@Override
+	public void send(DataPacket dataPacket) throws IOException {
+	    try {
+    		if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
+    			throw new IllegalStateException("Cannot send data because Transaction State is " + state);
+    		}
+    
+            if ( direction == TransferDirection.RECEIVE ) {
+                throw new IllegalStateException("Attempting to send data but started a RECEIVE Transaction");
+            }
+    
+    		if ( transfers > 0 ) {
+                ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
+            }
+    
+            logger.debug("{} Sending data to {}", this, peer);
+    
+    		final OutputStream out = new CheckedOutputStream(dos, crc);
+            codec.encode(dataPacket, out);
+            
+            // 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 ( compress ) {
+            	out.close();
+            }
+            
+            transfers++;
+            this.state = TransactionState.DATA_EXCHANGED;
+	    } catch (final Exception e) {
+	        error();
+	        throw e;
+	    }
+	}
+	
+	
+	@Override
+	public void cancel(final String explanation) throws IOException {
+		if ( state == TransactionState.TRANSACTION_CANCELED || state == TransactionState.TRANSACTION_COMPLETED || state == TransactionState.ERROR ) {
+			throw new IllegalStateException("Cannot cancel transaction because state is already " + state);
+		}
+
+		try {
+		    ResponseCode.CANCEL_TRANSACTION.writeResponse(dos, explanation == null ? "<No explanation given>" : explanation);
+		    state = TransactionState.TRANSACTION_CANCELED;
+		} catch (final IOException ioe) {
+		    error();
+		    throw ioe;
+		}
+	}
+	
+	
+	@Override
+	public void complete(boolean requestBackoff) throws IOException {
+	    try {
+    		if ( state != TransactionState.TRANSACTION_CONFIRMED ) {
+    			throw new IllegalStateException("Cannot complete transaction because state is " + state + 
+    					"; Transaction can only be completed when state is " + TransactionState.TRANSACTION_CONFIRMED);
+    		}
+    		
+    		if ( direction == TransferDirection.RECEIVE ) {
+    		    if ( transfers == 0 ) {
+    		        state = TransactionState.TRANSACTION_COMPLETED;
+    		        return;
+    		    }
+    		    
+                if ( requestBackoff ) {
+                    // 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);
+                }
+                
+                state = TransactionState.TRANSACTION_COMPLETED;
+            } else {
+                final Response transactionResponse;
+                try {
+                    transactionResponse = Response.read(dis);
+                } catch (final IOException e) {
+                    throw new IOException(this + " Failed to receive a response from " + peer + " when expecting a TransactionFinished Indicator. " +
+                            "It is unknown whether or not the peer successfully received/processed the data.", e);
+                }
+                
+                logger.debug("{} Received {} from {}", this, transactionResponse, peer);
+                if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
+                    peer.penalize(penaltyMillis);
+                } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
+                    throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
+                }
+                
+                state = TransactionState.TRANSACTION_COMPLETED;
+            }
+	    } catch (final Exception e) {
+	        error();
+	        throw e;
+	    }
+	}
+	
+	
+	@Override
+	public void confirm() throws IOException {
+	    try {
+	        if ( state == TransactionState.TRANSACTION_STARTED && !dataAvailable && direction == TransferDirection.RECEIVE ) {
+	            // client requested to receive data but no data available. no need to confirm.
+	            state = TransactionState.TRANSACTION_CONFIRMED;
+	            return;
+	        }
+	        
+    		if ( state != TransactionState.DATA_EXCHANGED ) {
+    			throw new IllegalStateException("Cannot confirm Transaction because state is " + state + 
+    					"; Transaction can only be confirmed when state is " + TransactionState.DATA_EXCHANGED );
+    		}
+    
+            if ( direction == TransferDirection.RECEIVE ) {
+                if ( dataAvailable ) {
+                    throw new IllegalStateException("Cannot complete transaction because the sender has already sent more data than client has consumed.");
+                }
+                
+                // 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);
+                final String calculatedCRC = String.valueOf(crc.getValue());
+                ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
+                
+                final Response confirmTransactionResponse;
+                try {
+                    confirmTransactionResponse = Response.read(dis);
+                } catch (final IOException ioe) {
+                    logger.error("Failed to receive response code from {} when expected confirmation of transaction", peer);
+                    throw ioe;
+                }
+                
+                logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
+                
+                switch (confirmTransactionResponse.getCode()) {
+                    case CONFIRM_TRANSACTION:
+                        break;
+                    case BAD_CHECKSUM:
+                        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");
+                }
+                
+                state = TransactionState.TRANSACTION_CONFIRMED;
+            } else {
+                logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
+                ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
+                
+                final String calculatedCRC = String.valueOf(crc.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();
+                    
+                    // CRC was not used before version 4
+                    if ( protocolVersion > 3 ) {
+                        if ( !receivedCRC.equals(calculatedCRC) ) {
+                            ResponseCode.BAD_CHECKSUM.writeResponse(dos);
+                            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);
+                }
+                
+                state = TransactionState.TRANSACTION_CONFIRMED;
+            }
+	    } catch (final Exception e) {
+	        error();
+	        throw e;
+	    }
+	}
+
+	@Override
+	public void error() {
+	    this.state = TransactionState.ERROR;
+	}
+	
+	@Override
+	public TransactionState getState() {
+		return state;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/PeerStatusCache.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java
new file mode 100644
index 0000000..b2dbdcd
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/RemoteNiFiUtils.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+
+import javax.net.ssl.SSLContext;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.nifi.web.api.dto.ControllerDTO;
+import org.apache.nifi.web.api.entity.ControllerEntity;
+import org.apache.nifi.web.util.WebUtils;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.ClientResponse.Status;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+
+/**
+ *
+ */
+public class RemoteNiFiUtils {
+
+    public static final String CONTROLLER_URI_PATH = "/controller";
+
+    private static final int CONNECT_TIMEOUT = 10000;
+    private static final int READ_TIMEOUT = 10000;
+    
+    private final Client client;
+    
+    public RemoteNiFiUtils(final SSLContext sslContext) {
+        this.client = getClient(sslContext);
+    }
+    
+
+    /**
+     * Gets the content at the specified URI.
+     *
+     * @param uri
+     * @param timeoutMillis
+     * @return
+     * @throws ClientHandlerException
+     * @throws UniformInterfaceException
+     */
+    public ClientResponse get(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
+        return get(uri, timeoutMillis, null);
+    }
+    
+    /**
+     * Gets the content at the specified URI using the given query parameters.
+     *
+     * @param uri
+     * @param timeoutMillis
+     * @param queryParams 
+     * @return
+     * @throws ClientHandlerException
+     * @throws UniformInterfaceException
+     */
+    public ClientResponse get(final URI uri, final int timeoutMillis, final Map<String, String> queryParams) throws ClientHandlerException, UniformInterfaceException {
+        // perform the request
+        WebResource webResource = client.resource(uri);
+        if ( queryParams != null ) {
+            for ( final Map.Entry<String, String> queryEntry : queryParams.entrySet() ) {
+                webResource = webResource.queryParam(queryEntry.getKey(), queryEntry.getValue());
+            }
+        }
+
+        webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
+        webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
+
+        return webResource.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    }
+
+    /**
+     * Performs a HEAD request to the specified URI.
+     *
+     * @param uri
+     * @param timeoutMillis
+     * @return
+     * @throws ClientHandlerException
+     * @throws UniformInterfaceException
+     */
+    public ClientResponse head(final URI uri, final int timeoutMillis) throws ClientHandlerException, UniformInterfaceException {
+        // perform the request
+        WebResource webResource = client.resource(uri);
+        webResource.setProperty(ClientConfig.PROPERTY_READ_TIMEOUT, timeoutMillis);
+        webResource.setProperty(ClientConfig.PROPERTY_CONNECT_TIMEOUT, timeoutMillis);
+        return webResource.head();
+    }
+
+    /**
+     * Gets a client based on the specified URI.
+     * 
+     * @param uri
+     * @return 
+     */
+    private Client getClient(final SSLContext sslContext) {
+        final Client client;
+        if (sslContext == null) {
+            client = WebUtils.createClient(null);
+        } else {
+            client = WebUtils.createClient(null, sslContext);
+        }
+
+        client.setReadTimeout(READ_TIMEOUT);
+        client.setConnectTimeout(CONNECT_TIMEOUT);
+
+        return client;
+    }
+    
+    
+    /**
+     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
+     * is not configured to use Site-to-Site transfers.
+     * 
+     * @param uri the base URI of the remote instance. This should include the path only to the nifi-api level, as well as the protocol, host, and port.
+     * @param timeoutMillis
+     * @return
+     * @throws IOException
+     */
+    public Integer getRemoteListeningPort(final String uri, final int timeoutMillis) throws IOException {
+    	try {
+			final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
+			return getRemoteListeningPort(uriObject, timeoutMillis);
+		} catch (URISyntaxException e) {
+			throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
+		}
+    }
+    
+    public String getRemoteRootGroupId(final String uri, final int timeoutMillis) throws IOException {
+        try {
+            final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
+            return getRemoteRootGroupId(uriObject, timeoutMillis);
+        } catch (URISyntaxException e) {
+            throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
+        }
+    }
+    
+    public String getRemoteInstanceId(final String uri, final int timeoutMillis) throws IOException {
+        try {
+            final URI uriObject = new URI(uri + CONTROLLER_URI_PATH);
+            return getController(uriObject, timeoutMillis).getInstanceId();
+        } catch (URISyntaxException e) {
+            throw new IOException("Unable to establish connection to remote host because URI is invalid: " + uri);
+        }
+    }
+    
+    /**
+     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance
+     * is not configured to use Site-to-Site transfers.
+     * 
+     * @param uri the full URI to fetch, including the path.
+     * @return
+     * @throws IOException
+     */
+    private Integer getRemoteListeningPort(final URI uri, final int timeoutMillis) throws IOException {
+    	return getController(uri, timeoutMillis).getRemoteSiteListeningPort();
+    }
+    
+    private String getRemoteRootGroupId(final URI uri, final int timeoutMillis) throws IOException {
+        return getController(uri, timeoutMillis).getId();
+    }
+    
+    public ControllerDTO getController(final URI uri, final int timeoutMillis) throws IOException {
+        final ClientResponse response = get(uri, timeoutMillis);
+        
+        if (Status.OK.getStatusCode() == response.getStatusInfo().getStatusCode()) {
+            final ControllerEntity entity = response.getEntity(ControllerEntity.class);
+            return entity.getController();
+        } else {
+            final String responseMessage = response.getEntity(String.class);
+            throw new IOException("Got HTTP response Code " + response.getStatusInfo().getStatusCode() + ": " + response.getStatusInfo().getReasonPhrase() + " with explanation: " + responseMessage);
+        }
+    }
+    
+    /**
+     * Issues a registration request on behalf of the current user.
+     * 
+     * @param baseApiUri 
+     * @return  
+     */
+    public ClientResponse issueRegistrationRequest(String baseApiUri) {
+        final URI uri = URI.create(String.format("%s/%s", baseApiUri, "/controller/users"));
+
+        // set up the query params
+        MultivaluedMapImpl entity = new MultivaluedMapImpl();
+        entity.add("justification", "A Remote instance of NiFi has attempted to create a reference to this NiFi. This action must be approved first.");
+        
+        // create the web resource
+        WebResource webResource = client.resource(uri);
+        
+        // get the client utils and make the request
+        return webResource.accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_FORM_URLENCODED).entity(entity).post(ClientResponse.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/StandardDataPacket.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/StandardDataPacket.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/StandardDataPacket.java
new file mode 100644
index 0000000..bd1b50c
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/StandardDataPacket.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.util;
+
+import java.io.InputStream;
+import java.util.Map;
+
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.stream.io.LimitingInputStream;
+import org.apache.nifi.stream.io.MinimumLengthInputStream;
+
+public class StandardDataPacket implements DataPacket {
+
+	private final Map<String, String> attributes;
+	private final InputStream stream;
+	private final long size;
+	
+	public StandardDataPacket(final Map<String, String> attributes, final InputStream stream, final long size) {
+		this.attributes = attributes;
+		this.stream = new MinimumLengthInputStream(new LimitingInputStream(stream, size), size);
+		this.size = size;
+	}
+	
+	public Map<String, String> getAttributes() {
+		return attributes;
+	}
+	
+	public InputStream getData() {
+		return stream;
+	}
+	
+	public long getSize() {
+		return size;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
new file mode 100644
index 0000000..d8899ea
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.remote.PeerStatus;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.remote.cluster.NodeInformation;
+import org.junit.Test;
+
+public class TestEndpointConnectionStatePool {
+
+    @Test
+    public void testFormulateDestinationListForOutput() throws IOException {
+        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
+        final List<NodeInformation> collection = new ArrayList<>();
+        collection.add(new NodeInformation("ShouldGetMedium", 1, 1111, true, 4096));
+        collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 10240));
+        collection.add(new NodeInformation("ShouldGetLittle", 3, 3333, true, 1024));
+        collection.add(new NodeInformation("ShouldGetMedium", 4, 4444, true, 4096));
+        collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
+
+        clusterNodeInfo.setNodeInformation(collection);
+        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        for ( final PeerStatus peerStatus : destinations ) {
+            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+        }
+    }
+    
+    @Test
+    public void testFormulateDestinationListForOutputHugeDifference() throws IOException {
+        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
+        final List<NodeInformation> collection = new ArrayList<>();
+        collection.add(new NodeInformation("ShouldGetLittle", 1, 1111, true, 500));
+        collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000));
+
+        clusterNodeInfo.setNodeInformation(collection);
+        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        for ( final PeerStatus peerStatus : destinations ) {
+            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+        }
+    }
+    
+    
+    
+    
+    @Test
+    public void testFormulateDestinationListForInputPorts() throws IOException {
+        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
+        final List<NodeInformation> collection = new ArrayList<>();
+        collection.add(new NodeInformation("ShouldGetMedium", 1, 1111, true, 4096));
+        collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 10240));
+        collection.add(new NodeInformation("ShouldGetLots", 3, 3333, true, 1024));
+        collection.add(new NodeInformation("ShouldGetMedium", 4, 4444, true, 4096));
+        collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
+
+        clusterNodeInfo.setNodeInformation(collection);
+        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        for ( final PeerStatus peerStatus : destinations ) {
+            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+        }
+    }
+    
+    @Test
+    public void testFormulateDestinationListForInputPortsHugeDifference() throws IOException {
+        final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation();
+        final List<NodeInformation> collection = new ArrayList<>();
+        collection.add(new NodeInformation("ShouldGetLots", 1, 1111, true, 500));
+        collection.add(new NodeInformation("ShouldGetLittle", 2, 2222, true, 50000));
+
+        clusterNodeInfo.setNodeInformation(collection);
+        final List<PeerStatus> destinations = EndpointConnectionStatePool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
+        for ( final PeerStatus peerStatus : destinations ) {
+            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
new file mode 100644
index 0000000..421d579
--- /dev/null
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.stream.io;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LimitingInputStream extends InputStream {
+
+    private final InputStream in;
+    private final long limit;
+    private long bytesRead = 0;
+
+    public LimitingInputStream(final InputStream in, final long limit) {
+        this.in = in;
+        this.limit = limit;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int val = in.read();
+        if (val > -1) {
+            bytesRead++;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(b.length, limit - bytesRead);
+
+        final int val = in.read(b, 0, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(len, limit - bytesRead);
+
+        final int val = in.read(b, off, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public long skip(final long n) throws IOException {
+        final long skipped = in.skip(Math.min(n, limit - bytesRead));
+        bytesRead += skipped;
+        return skipped;
+    }
+
+    @Override
+    public int available() throws IOException {
+        return in.available();
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public void mark(int readlimit) {
+        in.mark(readlimit);
+    }
+
+    @Override
+    public boolean markSupported() {
+        return in.markSupported();
+    }
+
+    @Override
+    public void reset() throws IOException {
+        in.reset();
+    }
+
+    public long getLimit() {
+    	return limit;
+    }
+}


[18/29] incubator-nifi git commit: NIFI-282: Fixed bug that caused load distribution across nodes in cluster not to work

Posted by ma...@apache.org.
NIFI-282: Fixed bug that caused load distribution across nodes in cluster not to work


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a7405b91
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a7405b91
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a7405b91

Branch: refs/heads/develop
Commit: a7405b915d059483135d706a7ce6a7b848b4c175
Parents: 2f60ddc
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 16 16:49:15 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 16 16:49:15 2015 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/remote/Peer.java  |   8 +-
 .../org/apache/nifi/remote/PeerDescription.java |  79 ++++++++++++
 .../java/org/apache/nifi/remote/PeerStatus.java |  32 ++---
 .../client/socket/EndpointConnectionPool.java   | 127 ++++++++++---------
 .../protocol/socket/SocketClientProtocol.java   |   5 +-
 .../socket/TestEndpointConnectionStatePool.java |   8 +-
 .../client/socket/TestSiteToSiteClient.java     |  41 +++---
 .../nifi/remote/SocketRemoteSiteListener.java   |   3 +-
 8 files changed, 198 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
index 3534f95..2428078 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
@@ -25,6 +25,7 @@ import org.apache.nifi.remote.protocol.CommunicationsSession;
 
 public class Peer implements Communicant {
 
+    private final PeerDescription description;
     private final CommunicationsSession commsSession;
     private final String url;
     private final String clusterUrl;
@@ -34,7 +35,8 @@ public class Peer implements Communicant {
     private final Map<String, Long> penaltyExpirationMap = new HashMap<>();
     private boolean closed = false;
 
-    public Peer(final CommunicationsSession commsSession, final String peerUrl, final String clusterUrl) {
+    public Peer(final PeerDescription description, final CommunicationsSession commsSession, final String peerUrl, final String clusterUrl) {
+        this.description = description;
         this.commsSession = commsSession;
         this.url = peerUrl;
         this.clusterUrl = clusterUrl;
@@ -48,6 +50,10 @@ public class Peer implements Communicant {
         }
     }
 
+    public PeerDescription getDescription() {
+        return description;
+    }
+    
     @Override
     public String getUrl() {
         return url;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerDescription.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerDescription.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerDescription.java
new file mode 100644
index 0000000..0e8e498
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerDescription.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;
+
+public class PeerDescription {
+    private final String hostname;
+    private final int port;
+    private final boolean secure;
+    
+    public PeerDescription(final String hostname, final int port, final boolean secure) {
+        this.hostname = hostname;
+        this.port = port;
+        this.secure = secure;
+    }
+
+    public String getHostname() {
+        return hostname;
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+    public boolean isSecure() {
+        return secure;
+    }
+
+    @Override
+    public String toString() {
+        return "PeerDescription[hostname=" + hostname + ", port=" + port + ", secure=" + secure + "]";
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((hostname == null) ? 0 : hostname.hashCode());
+        result = prime * result + port;
+        return result;
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+        
+        final PeerDescription other = (PeerDescription) obj;
+        if (hostname == null) {
+            if (other.hostname != null) {
+                return false;
+            }
+        } else if (!hostname.equals(other.hostname)) {
+            return false;
+        }
+        
+        return port == other.port;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
index d1cb076..b68ac33 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/PeerStatus.java
@@ -17,43 +17,31 @@
 package org.apache.nifi.remote;
 
 public class PeerStatus {
-
-    private final String hostname;
-    private final int port;
-    private final boolean secure;
+    private final PeerDescription description;
     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;
+    public PeerStatus(final PeerDescription description, final int numFlowFiles) {
+        this.description = description;
         this.numFlowFiles = numFlowFiles;
     }
 
-    public String getHostname() {
-        return hostname;
-    }
-
-    public int getPort() {
-        return port;
+    public PeerDescription getPeerDescription() {
+        return description;
     }
-
-    public boolean isSecure() {
-        return secure;
-    }
-
+    
     public int getFlowFileCount() {
         return numFlowFiles;
     }
 
     @Override
     public String toString() {
-        return "PeerStatus[hostname=" + hostname + ",port=" + port + ",secure=" + secure + ",flowFileCount=" + numFlowFiles + "]";
+        return "PeerStatus[hostname=" + description.getHostname() + ",port=" + description.getPort() + 
+                ",secure=" + description.isSecure() + ",flowFileCount=" + numFlowFiles + "]";
     }
 
     @Override
     public int hashCode() {
-        return 9824372 + hostname.hashCode() + port;
+        return 9824372 + description.getHostname().hashCode() + description.getPort() * 41;
     }
 
     @Override
@@ -67,6 +55,6 @@ public class PeerStatus {
         }
 
         final PeerStatus other = (PeerStatus) obj;
-        return port == other.port && hostname.equals(other.hostname);
+        return description.equals(other.getPeerDescription());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index f9a8a38..450daec 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -61,6 +61,7 @@ import javax.security.cert.CertificateNotYetValidException;
 
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.remote.Peer;
+import org.apache.nifi.remote.PeerDescription;
 import org.apache.nifi.remote.PeerStatus;
 import org.apache.nifi.remote.RemoteDestination;
 import org.apache.nifi.remote.RemoteResourceInitiator;
@@ -97,8 +98,8 @@ public class EndpointConnectionPool {
 
 	private static final Logger logger = LoggerFactory.getLogger(EndpointConnectionPool.class);
 	
-	private final BlockingQueue<EndpointConnection> connectionQueue = new LinkedBlockingQueue<>();
-    private final ConcurrentMap<PeerStatus, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
+	private final ConcurrentMap<PeerDescription, BlockingQueue<EndpointConnection>> connectionQueueMap = new ConcurrentHashMap<>();
+    private final ConcurrentMap<PeerDescription, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
     private final URI clusterUrl;
     private final String apiUri;
     
@@ -227,6 +228,23 @@ public class EndpointConnectionPool {
         SocketClientProtocol protocol = null;
         EndpointConnection connection;
         Peer peer = null;
+
+        logger.debug("{} getting next peer status", this);
+        final PeerStatus peerStatus = getNextPeerStatus(direction);
+        logger.debug("{} next peer status = {}", this, peerStatus);
+        if ( peerStatus == null ) {
+            return null;
+        }
+
+        final PeerDescription peerDescription = peerStatus.getPeerDescription();
+        BlockingQueue<EndpointConnection> connectionQueue = connectionQueueMap.get(peerStatus);
+        if ( connectionQueue == null ) {
+            connectionQueue = new LinkedBlockingQueue<>();
+            BlockingQueue<EndpointConnection> existing = connectionQueueMap.putIfAbsent(peerDescription, connectionQueue);
+            if ( existing != null ) {
+                connectionQueue = existing;
+            }
+        }
         
         final List<EndpointConnection> addBack = new ArrayList<>();
         try {
@@ -254,19 +272,12 @@ public class EndpointConnectionPool {
                     protocol = new SocketClientProtocol();
                     protocol.setDestination(new IdEnrichedRemoteDestination(remoteDestination, portId));
 
-                    logger.debug("{} getting next peer status", this);
-                    final PeerStatus peerStatus = getNextPeerStatus(direction);
-                    logger.debug("{} next peer status = {}", this, peerStatus);
-                    if ( peerStatus == null ) {
-                        return null;
-                    }
-
                     final long penalizationMillis = remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS);
                     try {
                         logger.debug("{} Establishing site-to-site connection with {}", this, peerStatus);
                         commsSession = establishSiteToSiteConnection(peerStatus);
                     } catch (final IOException ioe) {
-                        penalize(peerStatus, penalizationMillis);
+                        penalize(peerStatus.getPeerDescription(), penalizationMillis);
                         throw ioe;
                     }
                     
@@ -283,8 +294,8 @@ public class EndpointConnectionPool {
                         }
                     }
                 
-                    final String peerUrl = "nifi://" + peerStatus.getHostname() + ":" + peerStatus.getPort();
-                    peer = new Peer(commsSession, peerUrl, clusterUrl.toString());
+                    final String peerUrl = "nifi://" + peerDescription.getHostname() + ":" + peerDescription.getPort();
+                    peer = new Peer(peerDescription, commsSession, peerUrl, clusterUrl.toString());
     
                     // set properties based on config
                     if ( config != null ) {
@@ -371,6 +382,11 @@ public class EndpointConnectionPool {
     		return false;
     	}
     	
+    	final BlockingQueue<EndpointConnection> connectionQueue = connectionQueueMap.get(peer.getDescription());
+    	if ( connectionQueue == null ) {
+    	    return false;
+    	}
+    	
     	activeConnections.remove(endpointConnection);
     	if ( shutdown ) {
     	    terminate(endpointConnection);
@@ -381,14 +397,14 @@ public class EndpointConnectionPool {
     	}
     }
     
-    private void penalize(final PeerStatus status, final long penalizationMillis) {
-        Long expiration = peerTimeoutExpirations.get(status);
+    private void penalize(final PeerDescription peerDescription, final long penalizationMillis) {
+        Long expiration = peerTimeoutExpirations.get(peerDescription);
         if ( expiration == null ) {
             expiration = Long.valueOf(0L);
         }
         
         final long newExpiration = Math.max(expiration, System.currentTimeMillis() + penalizationMillis);
-        peerTimeoutExpirations.put(status, Long.valueOf(newExpiration));
+        peerTimeoutExpirations.put(peerDescription, Long.valueOf(newExpiration));
     }
     
     /**
@@ -396,19 +412,7 @@ public class EndpointConnectionPool {
      * @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);
-        penalize(status, penalizationMillis);
+        penalize(peer.getDescription(), penalizationMillis);
     }
     
     private void cleanup(final SocketClientProtocol protocol, final Peer peer) {
@@ -509,7 +513,8 @@ public class EndpointConnectionPool {
         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());
+            final PeerDescription description = peerStatus.getPeerDescription();
+            final NodeInformation nodeInfo = new NodeInformation(description.getHostname(), description.getPort(), 0, description.isSecure(), peerStatus.getFlowFileCount());
             nodeInfos.add(nodeInfo);
         }
         clusterNodeInfo.setNodeInformation(nodeInfos);
@@ -526,7 +531,7 @@ public class EndpointConnectionPool {
         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);
+                final PeerStatus equalizedStatus = new PeerStatus(status.getPeerDescription(), 1);
                 equalizedSet.add(equalizedStatus);
             }
 
@@ -543,8 +548,9 @@ public class EndpointConnectionPool {
             throw new IOException("Remote instance of NiFi is not configured to allow site-to-site communications");
         }
     	
+        final PeerDescription clusterPeerDescription = new PeerDescription(hostname, port, clusterUrl.toString().startsWith("https://"));
     	final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port);
-        final Peer peer = new Peer(commsSession, "nifi://" + hostname + ":" + port, clusterUrl.toString());
+        final Peer peer = new Peer(clusterPeerDescription, commsSession, "nifi://" + hostname + ":" + port, clusterUrl.toString());
         final SocketClientProtocol clientProtocol = new SocketClientProtocol();
         final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
         final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
@@ -602,7 +608,8 @@ public class EndpointConnectionPool {
              final OutputStream out = new BufferedOutputStream(fos)) {
 
             for (final PeerStatus status : statuses) {
-                final String line = status.getHostname() + ":" + status.getPort() + ":" + status.isSecure() + "\n";
+                final PeerDescription description = status.getPeerDescription();
+                final String line = description.getHostname() + ":" + description.getPort() + ":" + description.isSecure() + "\n";
                 out.write(line.getBytes(StandardCharsets.UTF_8));
             }
 
@@ -631,7 +638,7 @@ public class EndpointConnectionPool {
                 final int port = Integer.parseInt(splits[1]);
                 final boolean secure = Boolean.parseBoolean(splits[2]);
 
-                statuses.add(new PeerStatus(hostname, port, secure, 1));
+                statuses.add(new PeerStatus(new PeerDescription(hostname, port, secure), 1));
             }
         }
 
@@ -640,7 +647,8 @@ public class EndpointConnectionPool {
     
     
     private CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException {
-    	return establishSiteToSiteConnection(peerStatus.getHostname(), peerStatus.getPort());
+        final PeerDescription description = peerStatus.getPeerDescription();
+    	return establishSiteToSiteConnection(description.getHostname(), description.getPort());
     }
     
     private CommunicationsSession establishSiteToSiteConnection(final String hostname, final int port) throws IOException {
@@ -720,7 +728,8 @@ public class EndpointConnectionPool {
                     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());
+                        final PeerDescription description = new PeerDescription(nodeInfo.getHostname(), nodeInfo.getSiteToSitePort(), nodeInfo.isSiteToSiteSecure());
+                        status = new PeerStatus(description, nodeInfo.getTotalFlowFiles());
                         destinations.set(index, status);
                         break;
                     } else {
@@ -744,27 +753,29 @@ public class EndpointConnectionPool {
     
     
     private void cleanupExpiredSockets() {
-        final List<EndpointConnection> connections = new ArrayList<>();
-        
-        EndpointConnection connection;
-        while ((connection = connectionQueue.poll()) != null) {
-            // If the socket has not been used in 10 seconds, shut it down.
-            final long lastUsed = connection.getLastTimeUsed();
-            if ( lastUsed < System.currentTimeMillis() - idleExpirationMillis ) {
-                try {
-                    connection.getSocketClientProtocol().shutdown(connection.getPeer());
-                } catch (final Exception e) {
-                    logger.debug("Failed to shut down {} using {} due to {}", 
-                        new Object[] {connection.getSocketClientProtocol(), connection.getPeer(), e} );
+        for ( final BlockingQueue<EndpointConnection> connectionQueue : connectionQueueMap.values()) {
+            final List<EndpointConnection> connections = new ArrayList<>();
+            
+            EndpointConnection connection;
+            while ((connection = connectionQueue.poll()) != null) {
+                // If the socket has not been used in 10 seconds, shut it down.
+                final long lastUsed = connection.getLastTimeUsed();
+                if ( lastUsed < System.currentTimeMillis() - idleExpirationMillis ) {
+                    try {
+                        connection.getSocketClientProtocol().shutdown(connection.getPeer());
+                    } catch (final Exception e) {
+                        logger.debug("Failed to shut down {} using {} due to {}", 
+                            new Object[] {connection.getSocketClientProtocol(), connection.getPeer(), e} );
+                    }
+                    
+                    terminate(connection);
+                } else {
+                    connections.add(connection);
                 }
-                
-                terminate(connection);
-            } else {
-                connections.add(connection);
             }
+            
+            connectionQueue.addAll(connections);
         }
-        
-        connectionQueue.addAll(connections);
     }
     
     public void shutdown() {
@@ -775,10 +786,12 @@ public class EndpointConnectionPool {
        for ( final EndpointConnection conn : activeConnections ) {
            conn.getPeer().getCommunicationsSession().interrupt();
         }
-        
-        EndpointConnection state;
-        while ( (state = connectionQueue.poll()) != null)  {
-            cleanup(state.getSocketClientProtocol(), state.getPeer());
+
+        for ( final BlockingQueue<EndpointConnection> connectionQueue : connectionQueueMap.values() ) {
+            EndpointConnection state;
+            while ( (state = connectionQueue.poll()) != null)  {
+                cleanup(state.getSocketClientProtocol(), state.getPeer());
+            }
         }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
index 390f4fc..c3275ea 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@ -34,6 +34,7 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.remote.Peer;
+import org.apache.nifi.remote.PeerDescription;
 import org.apache.nifi.remote.PeerStatus;
 import org.apache.nifi.remote.RemoteDestination;
 import org.apache.nifi.remote.RemoteResourceInitiator;
@@ -117,7 +118,7 @@ public class SocketClientProtocol implements ClientProtocol {
         properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression));
         
         if ( destinationId != null ) {
-        	properties.put(HandshakeProperty.PORT_IDENTIFIER, destination.getIdentifier());
+        	properties.put(HandshakeProperty.PORT_IDENTIFIER, destinationId);
         }
         
         properties.put(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, String.valueOf(timeoutMillis) );
@@ -229,7 +230,7 @@ public class SocketClientProtocol implements ClientProtocol {
             final int port = dis.readInt();
             final boolean secure = dis.readBoolean();
             final int flowFileCount = dis.readInt();
-            peers.add(new PeerStatus(hostname, port, secure, flowFileCount));
+            peers.add(new PeerStatus(new PeerDescription(hostname, port, secure), flowFileCount));
         }
         
         logger.debug("{} Received {} Peer Statuses from {}", this, peers.size(), peer);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
index 275e40c..cb7af08 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java
@@ -41,7 +41,7 @@ public class TestEndpointConnectionStatePool {
         clusterNodeInfo.setNodeInformation(collection);
         final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+            System.out.println(peerStatus.getPeerDescription());
         }
     }
     
@@ -55,7 +55,7 @@ public class TestEndpointConnectionStatePool {
         clusterNodeInfo.setNodeInformation(collection);
         final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+            System.out.println(peerStatus.getPeerDescription());
         }
     }
     
@@ -75,7 +75,7 @@ public class TestEndpointConnectionStatePool {
         clusterNodeInfo.setNodeInformation(collection);
         final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+            System.out.println(peerStatus.getPeerDescription());
         }
     }
     
@@ -89,7 +89,7 @@ public class TestEndpointConnectionStatePool {
         clusterNodeInfo.setNodeInformation(collection);
         final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
         for ( final PeerStatus peerStatus : destinations ) {
-            System.out.println(peerStatus.getHostname() + ":" + peerStatus.getPort());
+            System.out.println(peerStatus.getPeerDescription());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
index bb16a34..0f48b03 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
@@ -43,26 +43,31 @@ public class TestSiteToSiteClient {
         final SiteToSiteClient client = new SiteToSiteClient.Builder()
             .url("http://localhost:8080/nifi")
             .portName("cba")
-            .requestBatchCount(1)
+            .requestBatchCount(10)
             .build();
         
         try {
-            final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
-            Assert.assertNotNull(transaction);
-            
-            final DataPacket packet = transaction.receive();
-            Assert.assertNotNull(packet);
-            
-            final InputStream in = packet.getData();
-            final long size = packet.getSize();
-            final byte[] buff = new byte[(int) size];
-            
-            StreamUtils.fillBuffer(in, buff);
-            
-            Assert.assertNull(transaction.receive());
-            
-            transaction.confirm();
-            transaction.complete();
+            for (int i=0; i < 1000; i++) {
+                final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
+                Assert.assertNotNull(transaction);
+                
+                DataPacket packet;
+                while (true) {
+                    packet = transaction.receive();
+                    if ( packet == null ) {
+                        break;
+                    }
+
+                    final InputStream in = packet.getData();
+                    final long size = packet.getSize();
+                    final byte[] buff = new byte[(int) size];
+                    
+                    StreamUtils.fillBuffer(in, buff);
+                }
+                
+                transaction.confirm();
+                transaction.complete();
+            }
         } finally {
             client.close();
         }
@@ -70,7 +75,7 @@ public class TestSiteToSiteClient {
     
     
     @Test
-    //@Ignore("For local testing only; not really a unit test but a manual test")
+    @Ignore("For local testing only; not really a unit test but a manual test")
     public void testSend() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a7405b91/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
index 8a4839b..493d1fe 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
@@ -206,7 +206,8 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
                             	protocol.setRootProcessGroup(rootGroup.get());
                           	    protocol.setNodeInformant(nodeInformant);
                             	
-                            	peer = new Peer(commsSession, peerUri, "nifi://localhost:" + getPort());
+                          	    final PeerDescription description = new PeerDescription("localhost", getPort(), sslContext != null);
+                            	peer = new Peer(description, commsSession, peerUri, "nifi://localhost:" + getPort());
                             	LOG.debug("Handshaking....");
                             	protocol.handshake(peer);
                             	


[12/29] incubator-nifi git commit: Refactored client and add javadocs

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index 1e33e1f..da9d027 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -18,6 +18,7 @@ package org.apache.nifi.remote;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -25,8 +26,6 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 
 import javax.net.ssl.SSLContext;
 
@@ -34,28 +33,30 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.ProcessScheduler;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
 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.EndpointConnection;
-import org.apache.nifi.remote.client.socket.EndpointConnectionPool;
-import org.apache.nifi.remote.codec.FlowFileCodec;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.remote.client.SiteToSiteClient;
 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.protocol.ClientProtocol;
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-import org.apache.nifi.remote.protocol.socket.SocketClientProtocol;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.remote.util.StandardDataPacket;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+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;
 
 public class StandardRemoteGroupPort extends RemoteGroupPort {
+    private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
     public static final String USER_AGENT = "NiFi-Site-to-Site";
     public static final String CONTENT_TYPE = "application/octet-stream";
     
@@ -71,11 +72,8 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     private final SSLContext sslContext;
     private final TransferDirection transferDirection;
     
-    private final AtomicReference<EndpointConnectionPool> connectionPoolRef = new AtomicReference<>();
+    private final AtomicReference<SiteToSiteClient> clientRef = new AtomicReference<>();
     
-    private final Set<CommunicationsSession> activeCommsChannels = new HashSet<>();
-    private final Lock interruptLock = new ReentrantLock();
-    private boolean shutdown = false;   // guarded by codecLock
     
     public StandardRemoteGroupPort(final String id, final String name, final ProcessGroup processGroup, final RemoteProcessGroup remoteGroup, 
             final TransferDirection direction, final ConnectableType type, final SSLContext sslContext, final ProcessScheduler scheduler) {
@@ -112,16 +110,14 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     @Override
     public void shutdown() {
     	super.shutdown();
-        interruptLock.lock();
-        try {
-            this.shutdown = true;
-        } finally {
-            interruptLock.unlock();
-        }
         
-        final EndpointConnectionPool pool = connectionPoolRef.get();
-        if ( pool != null ) {
-            pool.shutdown();
+        final SiteToSiteClient client = clientRef.get();
+        if ( client != null ) {
+            try {
+                client.close();
+            } catch (final IOException ioe) {
+                logger.warn("Failed to properly shutdown Site-to-Site Client due to {}", ioe);
+            }
         }
     }
     
@@ -129,17 +125,14 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     public void onSchedulingStart() {
         super.onSchedulingStart();
         
-        interruptLock.lock();
-        try {
-            this.shutdown = false;
-        } finally {
-            interruptLock.unlock();
-        }
-        
-        final EndpointConnectionPool connectionPool = new EndpointConnectionPool(remoteGroup.getTargetUri().toString(), 
-                remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS), 
-                sslContext, remoteGroup.getEventReporter(), getPeerPersistenceFile(getIdentifier()));
-        connectionPoolRef.set(connectionPool);
+        final SiteToSiteClient client = new SiteToSiteClient.Builder()
+            .url(remoteGroup.getTargetUri().toString())
+            .portIdentifier(getIdentifier())
+            .sslContext(sslContext)
+            .eventReporter(remoteGroup.getEventReporter())
+            .peerPersistenceFile(getPeerPersistenceFile(getIdentifier()))
+            .build();
+        clientRef.set(client);
     }
     
     
@@ -157,10 +150,10 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         
         String url = getRemoteProcessGroup().getTargetUri().toString();
         
-        final EndpointConnectionPool connectionPool = connectionPoolRef.get();
-        final EndpointConnection connection;
+        final SiteToSiteClient client = clientRef.get();
+        final Transaction transaction;
         try {
-        	connection = connectionPool.getEndpointConnection(this, transferDirection);
+        	transaction = client.createTransaction(transferDirection);
         } catch (final PortNotRunningException e) {
             context.yield();
             this.targetRunning.set(false);
@@ -186,95 +179,36 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             return;
         }
         
-        if ( connection == null ) {
-            logger.debug("{} Unable to determine the next peer to communicate with; all peers must be penalized, so yielding context", this);
+        if ( transaction == null ) {
+            logger.debug("{} Unable to create transaction to communicate with; all peers must be penalized, so yielding context", this);
             context.yield();
             return;
         }
-        
-        FlowFileCodec codec = connection.getCodec();
-        SocketClientProtocol protocol = connection.getSocketClientProtocol();
-        final Peer peer = connection.getPeer();
-        url = peer.getUrl();
-        
+
         try {
-            interruptLock.lock();
-            try {
-                if ( shutdown ) {
-                    peer.getCommunicationsSession().interrupt();
-                }
-                
-                activeCommsChannels.add(peer.getCommunicationsSession());
-            } finally {
-                interruptLock.unlock();
-            }
-            
             if ( getConnectableType() == ConnectableType.REMOTE_INPUT_PORT ) {
-                transferFlowFiles(peer, protocol, context, session, codec);
+                transferFlowFiles(transaction, context, session);
             } else {
-                final int numReceived = receiveFlowFiles(peer, protocol, context, session, codec);
+                final int numReceived = receiveFlowFiles(transaction, context, session);
                 if ( numReceived == 0 ) {
                     context.yield();
                 }
             }
 
-            interruptLock.lock();
-            try {
-                if ( shutdown ) {
-                    peer.getCommunicationsSession().interrupt();
-                }
-                
-                activeCommsChannels.remove(peer.getCommunicationsSession());
-            } finally {
-                interruptLock.unlock();
-            }
-
             session.commit();
-            
-            connection.setLastTimeUsed();
-            connectionPool.offer(connection);
-        } catch (final TransmissionDisabledException e) {
-            cleanup(protocol, peer);
-            session.rollback();
         } catch (final Exception e) {
-            connectionPool.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);
+            final String message = String.format("%s failed to communicate with remote NiFi instance due to %s", this, e.toString());
+            logger.error("{} failed to communicate with remote NiFi instance due to {}", this, e.toString());
             if ( logger.isDebugEnabled() ) {
                 logger.error("", e);
             }
             
-            cleanup(protocol, peer);
-            
             remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
             session.rollback();
         }
     }
 
     
-    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) {
-            }
-        }
-    }
-    
     @Override
     public String getYieldPeriod() {
         // delegate yield duration to remote process group
@@ -282,12 +216,129 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     }
     
     
-    private int transferFlowFiles(final Peer peer, final ClientProtocol protocol, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
-        return protocol.transferFlowFiles(peer, context, session, codec);
+    private int transferFlowFiles(final Transaction transaction, final ProcessContext context, final ProcessSession session) throws IOException, ProtocolException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return 0;
+        }
+
+        try {
+            final String userDn = transaction.getCommunicant().getDistinguishedName();
+            final long startSendingNanos = System.nanoTime();
+            final StopWatch stopWatch = new StopWatch(true);
+            long bytesSent = 0L;
+            
+            final Set<FlowFile> flowFilesSent = new HashSet<>();
+            boolean continueTransaction = true;
+            while (continueTransaction) {
+                final long startNanos = System.nanoTime();
+                // call codec.encode within a session callback so that we have the InputStream to read the FlowFile
+                final FlowFile toWrap = flowFile;
+                session.read(flowFile, new InputStreamCallback() {
+                    @Override
+                    public void process(final InputStream in) throws IOException {
+                        final DataPacket dataPacket = new StandardDataPacket(toWrap.getAttributes(), in, toWrap.getSize());
+                        transaction.send(dataPacket);
+                    }
+                });
+                
+                final long transferNanos = System.nanoTime() - startNanos;
+                final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+                
+                flowFilesSent.add(flowFile);
+                bytesSent += flowFile.getSize();
+                logger.debug("{} Sent {} to {}", this, flowFile, transaction.getCommunicant().getUrl());
+                
+                final String transitUri = transaction.getCommunicant().getUrl() + "/" + flowFile.getAttribute(CoreAttributes.UUID.key());
+                session.getProvenanceReporter().send(flowFile, transitUri, "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);
+            }
+            
+            transaction.confirm();
+            
+            // 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();
+            transaction.complete();
+            
+            final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
+            logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
+                this, flowFileDescription, dataSize, transaction.getCommunicant().getUrl(), uploadMillis, uploadDataRate});
+            
+            return flowFilesSent.size();
+        } catch (final Exception e) {
+            session.rollback();
+            throw e;
+        }
+
+        
     }
     
-    private int receiveFlowFiles(final Peer peer, final ClientProtocol protocol, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
-        return protocol.receiveFlowFiles(peer, context, session, codec);
+    private int receiveFlowFiles(final Transaction transaction, final ProcessContext context, final ProcessSession session) throws IOException, ProtocolException {
+        final String userDn = transaction.getCommunicant().getDistinguishedName();
+        
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<FlowFile> flowFilesReceived = new HashSet<>();
+        long bytesReceived = 0L;
+        
+        while (true) {
+            final long start = System.nanoTime();
+            final DataPacket dataPacket = transaction.receive();
+            if ( dataPacket == null ) {
+                break;
+            }
+            
+            FlowFile flowFile = session.create();
+            flowFile = session.putAllAttributes(flowFile, dataPacket.getAttributes());
+            flowFile = session.importFrom(dataPacket.getData(), flowFile);
+            final long receiveNanos = System.nanoTime() - start;
+            
+            String sourceFlowFileIdentifier = dataPacket.getAttributes().get(CoreAttributes.UUID.key());
+            if ( sourceFlowFileIdentifier == null ) {
+                sourceFlowFileIdentifier = "<Unknown Identifier>";
+            }
+            
+            final String transitUri = transaction.getCommunicant().getUrl() + sourceFlowFileIdentifier;
+            session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, 
+                    "Remote DN=" + userDn, TimeUnit.NANOSECONDS.toMillis(receiveNanos));
+
+            session.transfer(flowFile, Relationship.ANONYMOUS);
+            bytesReceived += dataPacket.getSize();
+        }
+
+        // Confirm that what we received was the correct data.
+        transaction.confirm();
+        
+        // Commit the session so that we have persisted the data
+        session.commit();
+
+        transaction.complete();
+
+        if ( !flowFilesReceived.isEmpty() ) {
+            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, transaction.getCommunicant().getUrl(), uploadMillis, uploadDataRate });
+        }
+        
+        return flowFilesReceived.size();
     }
 
     @Override


[05/29] incubator-nifi git commit: NIFI-282: Bug fixes; documentation improvements; removed code marked as 'FOR TESTING'

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
new file mode 100644
index 0000000..8c23e28
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionStatePool.java
@@ -0,0 +1,835 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
+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 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.client.SiteToSiteClientConfig;
+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.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.remote.util.RemoteNiFiUtils;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.web.api.dto.ControllerDTO;
+import org.apache.nifi.web.api.dto.PortDTO;
+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";
+    public static final long REMOTE_REFRESH_MILLIS = TimeUnit.MILLISECONDS.convert(10, TimeUnit.MINUTES);
+
+    private static final long PEER_CACHE_MILLIS = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
+
+	private static final Logger logger = LoggerFactory.getLogger(EndpointConnectionStatePool.class);
+	
+	private final BlockingQueue<EndpointConnectionState> connectionStateQueue = new LinkedBlockingQueue<>();
+    private final ConcurrentMap<PeerStatus, Long> peerTimeoutExpirations = new ConcurrentHashMap<>();
+    private final URI clusterUrl;
+    private final String apiUri;
+    
+    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;
+    private final ScheduledExecutorService taskExecutor;
+    
+    private final ReadWriteLock listeningPortRWLock = new ReentrantReadWriteLock();
+    private final Lock remoteInfoReadLock = listeningPortRWLock.readLock();
+    private final Lock remoteInfoWriteLock = listeningPortRWLock.writeLock();
+    private Integer siteToSitePort;
+    private Boolean siteToSiteSecure;
+    private long remoteRefreshTime;
+    private final Map<String, String> inputPortMap = new HashMap<>();	// map input port name to identifier
+    private final Map<String, String> outputPortMap = new HashMap<>();	// map output port name to identifier
+    
+    private volatile int commsTimeout;
+
+    public EndpointConnectionStatePool(final String clusterUrl, final int commsTimeoutMillis, final EventReporter eventReporter, final File persistenceFile) {
+    	this(clusterUrl, commsTimeoutMillis, null, eventReporter, persistenceFile);
+    }
+    
+    public EndpointConnectionStatePool(final String clusterUrl, final int commsTimeoutMillis, final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) {
+    	try {
+    		this.clusterUrl = new URI(clusterUrl);
+    	} catch (final URISyntaxException e) {
+    		throw new IllegalArgumentException("Invalid Cluster URL: " + clusterUrl);
+    	}
+    	
+    	// Trim the trailing /
+        String uriPath = this.clusterUrl.getPath();
+        if (uriPath.endsWith("/")) {
+            uriPath = uriPath.substring(0, uriPath.length() - 1);
+        }
+        apiUri = this.clusterUrl.getScheme() + "://" + this.clusterUrl.getHost() + ":" + this.clusterUrl.getPort() + uriPath + "-api";
+        
+    	this.sslContext = sslContext;
+    	this.peersFile = persistenceFile;
+    	this.eventReporter = eventReporter;
+    	this.commsTimeout = commsTimeoutMillis;
+    	
+    	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;
+    	}
+
+    	// Initialize a scheduled executor and run some maintenance tasks in the background to kill off old, unused
+    	// connections and keep our list of peers up-to-date.
+    	taskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
+    		private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+    		
+			@Override
+			public Thread newThread(final Runnable r) {
+				final Thread thread = defaultFactory.newThread(r);
+				thread.setName("NiFi Site-to-Site Connection Pool Maintenance");
+				return thread;
+			}
+    	});
+
+    	taskExecutor.scheduleWithFixedDelay(new Runnable() {
+			@Override
+			public void run() {
+				refreshPeers();
+			}
+    	}, 0, 5, TimeUnit.SECONDS);
+
+    	taskExecutor.scheduleWithFixedDelay(new Runnable() {
+			@Override
+			public void run() {
+				cleanupExpiredSockets();
+			}
+    	}, 5, 5, TimeUnit.SECONDS);
+    }
+    
+    
+    public EndpointConnectionState getEndpointConnectionState(final RemoteDestination remoteDestination, final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
+        return getEndpointConnectionState(remoteDestination, direction, null);
+    }
+    
+    
+    
+    public EndpointConnectionState getEndpointConnectionState(final RemoteDestination remoteDestination, final TransferDirection direction, final SiteToSiteClientConfig config) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
+    	//
+        // Attempt to get a connection state that already exists for this URL.
+        //
+        FlowFileCodec codec = null;
+        CommunicationsSession commsSession = null;
+        SocketClientProtocol protocol = null;
+        EndpointConnectionState connectionState;
+        Peer peer = null;
+        
+        final List<EndpointConnectionState> addBack = new ArrayList<>();
+        try {
+            do {
+                final PeerStatus peerStatus = getNextPeerStatus(direction);
+                if ( peerStatus == null ) {
+                	return null;
+                }
+    
+                connectionState = connectionStateQueue.poll();
+                logger.debug("{} Connection State for {} = {}", this, clusterUrl, connectionState);
+                
+                if ( connectionState == null && !addBack.isEmpty() ) {
+                    // all available connections have been penalized.
+                    return null;
+                }
+                
+                if ( connectionState != null && connectionState.getPeer().isPenalized() ) {
+                    // we have a connection, but it's penalized. We want to add it back to the queue
+                    // when we've found one to use.
+                    addBack.add(connectionState);
+                    continue;
+                }
+                
+                // 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.toString());
+    
+                    // set properties based on config
+                    if ( config != null ) {
+                        protocol.setTimeout((int) config.getTimeout(TimeUnit.MILLISECONDS));
+                        protocol.setPreferredBatchCount(config.getPreferredBatchCount());
+                        protocol.setPreferredBatchSize(config.getPreferredBatchSize());
+                        protocol.setPreferredBatchDuration(config.getPreferredBatchDuration(TimeUnit.MILLISECONDS));
+                    }
+                    
+                    // 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;
+                    
+                    if ( commsTimeout > 0L && millisSinceLastUse >= commsTimeout ) {
+                        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 );
+        } finally {
+            if ( !addBack.isEmpty() ) {
+                connectionStateQueue.addAll(addBack);
+            }
+        }
+        
+        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;
+    	}
+    	
+    	return connectionStateQueue.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);
+                    }
+                    
+                    if ( eventReporter != null ) {
+                    	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, 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);
+    }
+    
+    
+    private 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() throws IOException, HandshakeException, UnknownPortException, PortNotRunningException {
+    	final String hostname = clusterUrl.getHost();
+        final int port = getSiteToSitePort();
+    	
+    	final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port);
+        final Peer peer = new Peer(commsSession, "nifi://" + hostname + ":" + port, clusterUrl.toString());
+        final SocketClientProtocol clientProtocol = new SocketClientProtocol();
+        final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+        final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+        RemoteResourceInitiator.initiateResourceNegotiation(clientProtocol, dis, dos);
+
+        clientProtocol.setTimeout(commsTimeout);
+        clientProtocol.handshake(peer, null);
+        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;
+    }
+    
+    
+    private CommunicationsSession establishSiteToSiteConnection(final PeerStatus peerStatus) throws IOException {
+    	return establishSiteToSiteConnection(peerStatus.getHostname(), peerStatus.getPort());
+    }
+    
+    private CommunicationsSession establishSiteToSiteConnection(final String hostname, final int port) throws IOException {
+    	if ( siteToSiteSecure == null ) {
+    		throw new IOException("Remote NiFi instance " + clusterUrl + " is not currently configured to accept site-to-site connections");
+    	}
+    	
+        final String destinationUri = "nifi://" + hostname + ":" + port;
+
+        CommunicationsSession commsSession = null;
+        try {
+	        if ( siteToSiteSecure ) {
+	            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;
+    }
+    
+    
+    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;
+    }
+    
+    
+    private void cleanupExpiredSockets() {
+        final List<EndpointConnectionState> states = new ArrayList<>();
+        
+        EndpointConnectionState state;
+        while ((state = connectionStateQueue.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);
+            }
+        }
+        
+        connectionStateQueue.addAll(states);
+    }
+    
+    public void shutdown() {
+    	taskExecutor.shutdown();
+    	peerTimeoutExpirations.clear();
+            
+        for ( final CommunicationsSession commsSession : activeCommsChannels ) {
+            commsSession.interrupt();
+        }
+        
+        EndpointConnectionState state;
+        while ( (state = connectionStateQueue.poll()) != null)  {
+            cleanup(state.getSocketClientProtocol(), state.getPeer());
+        }
+    }
+    
+    public void terminate(final EndpointConnectionState state) {
+        cleanup(state.getSocketClientProtocol(), state.getPeer());
+    }
+    
+    private void refreshPeers() {
+        final PeerStatusCache existingCache = peerStatusCache;
+        if (existingCache != null && (existingCache.getTimestamp() + PEER_CACHE_MILLIS > System.currentTimeMillis())) {
+            return;
+        }
+
+        try {
+            final Set<PeerStatus> statuses = fetchRemotePeerStatuses();
+            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);
+            }
+        }
+    }
+    
+    
+    public String getInputPortIdentifier(final String portName) throws IOException {
+        return getPortIdentifier(portName, inputPortMap);
+    }
+    
+    public String getOutputPortIdentifier(final String portName) throws IOException {
+    	return getPortIdentifier(portName, outputPortMap);
+    }
+    
+    
+    private String getPortIdentifier(final String portName, final Map<String, String> portMap) throws IOException {
+    	String identifier;
+    	remoteInfoReadLock.lock();
+        try {
+        	identifier = portMap.get(portName);
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+        
+        if ( identifier != null ) {
+        	return identifier;
+        }
+        
+        refreshRemoteInfo();
+
+    	remoteInfoReadLock.lock();
+        try {
+        	return portMap.get(portName);
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+    }
+    
+    
+    private ControllerDTO refreshRemoteInfo() throws IOException {
+    	final boolean webInterfaceSecure = clusterUrl.toString().startsWith("https");
+        final RemoteNiFiUtils utils = new RemoteNiFiUtils(webInterfaceSecure ? sslContext : null);
+		final ControllerDTO controller = utils.getController(URI.create(apiUri + "/controller"), commsTimeout);
+        
+        remoteInfoWriteLock.lock();
+        try {
+            this.siteToSitePort = controller.getRemoteSiteListeningPort();
+            this.siteToSiteSecure = controller.isSiteToSiteSecure();
+            
+            inputPortMap.clear();
+            for (final PortDTO inputPort : controller.getInputPorts()) {
+            	inputPortMap.put(inputPort.getName(), inputPort.getId());
+            }
+            
+            outputPortMap.clear();
+            for ( final PortDTO outputPort : controller.getOutputPorts()) {
+            	outputPortMap.put(outputPort.getName(), outputPort.getId());
+            }
+            
+            this.remoteRefreshTime = System.currentTimeMillis();
+        } finally {
+        	remoteInfoWriteLock.unlock();
+        }
+        
+        return controller;
+    }
+    
+    /**
+     * @return the port that the remote instance is listening on for
+     * site-to-site communication, or <code>null</code> if the remote instance
+     * is not configured to allow site-to-site communications.
+     *
+     * @throws IOException if unable to communicate with the remote instance
+     */
+    private Integer getSiteToSitePort() throws IOException {
+        Integer listeningPort;
+        remoteInfoReadLock.lock();
+        try {
+            listeningPort = this.siteToSitePort;
+            if (listeningPort != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
+                return listeningPort;
+            }
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+
+        final ControllerDTO controller = refreshRemoteInfo();
+        listeningPort = controller.getRemoteSiteListeningPort();
+
+        return listeningPort;
+    }
+ 
+    /**
+     * Returns {@code true} if the remote instance is configured for secure site-to-site communications,
+     * {@code false} otherwise.
+     * 
+     * @return
+     * @throws IOException
+     */
+    public boolean isSecure() throws IOException {
+        remoteInfoReadLock.lock();
+        try {
+            final Boolean secure = this.siteToSiteSecure;
+            if (secure != null && this.remoteRefreshTime > System.currentTimeMillis() - REMOTE_REFRESH_MILLIS) {
+                return secure;
+            }
+        } finally {
+        	remoteInfoReadLock.unlock();
+        }
+
+        final ControllerDTO controller = refreshRemoteInfo();
+        return controller.isSiteToSiteSecure();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
new file mode 100644
index 0000000..0494d04
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.remote.RemoteDestination;
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+import org.apache.nifi.remote.client.SiteToSiteClientConfig;
+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;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.util.ObjectHolder;
+
+public class SocketClient implements SiteToSiteClient {
+    private final SiteToSiteClientConfig config;
+	private final EndpointConnectionStatePool pool;
+	private final boolean compress;
+	private final String portName;
+	private final long penalizationNanos;
+	private volatile String portIdentifier;
+	
+	public SocketClient(final SiteToSiteClientConfig config) {
+		pool = new EndpointConnectionStatePool(config.getUrl(), (int) config.getTimeout(TimeUnit.MILLISECONDS), 
+				config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile());
+		
+		this.config = config;
+		this.compress = config.isUseCompression();
+		this.portIdentifier = config.getPortIdentifier();
+		this.portName = config.getPortName();
+		this.penalizationNanos = config.getPenalizationPeriod(TimeUnit.NANOSECONDS);
+	}
+	
+	@Override
+	public SiteToSiteClientConfig getConfig() {
+	    return config;
+	}
+	
+	@Override
+	public boolean isSecure() throws IOException {
+		return pool.isSecure();
+	}
+	
+	private String getPortIdentifier(final TransferDirection direction) throws IOException {
+		final String id = this.portIdentifier;
+		if ( id != null ) {
+			return id;
+		}
+		
+		if ( direction == TransferDirection.SEND ) {
+			return pool.getInputPortIdentifier(this.portName);
+		} else {
+			return pool.getOutputPortIdentifier(this.portName);
+		}
+	}
+	
+	
+	@Override
+	public Transaction createTransaction(final TransferDirection direction) throws IOException {
+		final String portId = getPortIdentifier(TransferDirection.SEND);
+		
+		if ( portId == null ) {
+			throw new IOException("Could not find Port with name " + portName + " for remote NiFi instance");
+		}
+		
+		final RemoteDestination remoteDestination = new RemoteDestination() {
+			@Override
+			public String getIdentifier() {
+				return portId;
+			}
+
+			@Override
+			public long getYieldPeriod(final TimeUnit timeUnit) {
+				return timeUnit.convert(penalizationNanos, TimeUnit.NANOSECONDS);
+			}
+
+			@Override
+			public boolean isUseCompression() {
+				return compress;
+			}
+		};
+		
+		final EndpointConnectionState connectionState;
+		try {
+			connectionState = pool.getEndpointConnectionState(remoteDestination, direction);
+		} catch (final ProtocolException | HandshakeException | PortNotRunningException | UnknownPortException e) {
+			throw new IOException(e);
+		}
+		
+		final Transaction transaction = connectionState.getSocketClientProtocol().startTransaction(
+				connectionState.getPeer(), connectionState.getCodec(), direction);
+		
+		// Wrap the transaction in a new one that will return the EndpointConnectionState back to the pool whenever
+		// the transaction is either completed or canceled.
+		final ObjectHolder<EndpointConnectionState> connectionStateRef = new ObjectHolder<>(connectionState);
+		return new Transaction() {
+			@Override
+			public void confirm() throws IOException {
+				transaction.confirm();
+			}
+
+			@Override
+			public void complete(final boolean requestBackoff) throws IOException {
+				try {
+					transaction.complete(requestBackoff);
+				} finally {
+				    final EndpointConnectionState state = connectionStateRef.get();
+				    if ( state != null ) {
+				        pool.offer(connectionState);
+				        connectionStateRef.set(null);
+				    }
+				}
+			}
+
+			@Override
+			public void cancel(final String explanation) throws IOException {
+				try {
+					transaction.cancel(explanation);
+				} finally {
+                    final EndpointConnectionState state = connectionStateRef.get();
+                    if ( state != null ) {
+                        pool.terminate(connectionState);
+                        connectionStateRef.set(null);
+                    }
+				}
+			}
+
+			@Override
+			public void error() {
+			    try {
+			        transaction.error();
+			    } finally {
+                    final EndpointConnectionState state = connectionStateRef.get();
+                    if ( state != null ) {
+                        pool.terminate(connectionState);
+                        connectionStateRef.set(null);
+                    }
+			    }
+			}
+			
+			@Override
+			public void send(final DataPacket dataPacket) throws IOException {
+				transaction.send(dataPacket);
+			}
+
+			@Override
+			public DataPacket receive() throws IOException {
+				return transaction.receive();
+			}
+
+			@Override
+			public TransactionState getState() throws IOException {
+				return transaction.getState();
+			}
+			
+		};
+	}
+
+	
+	@Override
+	public void close() throws IOException {
+		pool.shutdown();
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/AdaptedNodeInformation.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/ClusterNodeInformation.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformation.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/cluster/NodeInformationAdapter.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
new file mode 100644
index 0000000..1380e1b
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/FlowFileCodec.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.remote.VersionedRemoteResource;
+import org.apache.nifi.remote.exception.ProtocolException;
+import org.apache.nifi.remote.exception.TransmissionDisabledException;
+import org.apache.nifi.remote.protocol.DataPacket;
+
+/**
+ * <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 DataPacket and its content as a single stream of data and writes
+     * that stream to the output.
+     *
+     * @param dataPacket the data to serialize
+     * @param outStream the stream to write the data to
+     *
+     * @throws IOException if there is a communications issue
+     * @throws TransmissionDisabledException if a user terminates the connection
+     */
+    void encode(DataPacket dataPacket, OutputStream outStream) throws IOException, TransmissionDisabledException;
+
+    /**
+     * Decodes the contents of the InputStream, interpreting the data to
+     * determine the next DataPacket's attributes and content.
+     *
+     * @param stream an InputStream containing DataPacket's content and attributes
+     *
+     * @return the DataPacket that was created, or <code>null</code> if the stream
+     * was out of data
+     *
+     * @throws IOException
+     * @throws ProtocolException if the input is malformed
+     * @throws TransmissionDisabledException if a user terminates the connection
+     */
+    DataPacket decode(InputStream stream) throws IOException, ProtocolException, TransmissionDisabledException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
new file mode 100644
index 0000000..6fd92de
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.remote.StandardVersionNegotiator;
+import org.apache.nifi.remote.VersionNegotiator;
+import org.apache.nifi.remote.exception.ProtocolException;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.remote.util.StandardDataPacket;
+import org.apache.nifi.stream.io.StreamUtils;
+
+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 void encode(final DataPacket dataPacket, final OutputStream encodedOut) throws IOException {
+        final DataOutputStream out = new DataOutputStream(encodedOut);
+        
+        final Map<String, String> attributes = dataPacket.getAttributes();
+        out.writeInt(attributes.size());
+        for ( final Map.Entry<String, String> entry : attributes.entrySet() ) {
+            writeString(entry.getKey(), out);
+            writeString(entry.getValue(), out);
+        }
+        
+        out.writeLong(dataPacket.getSize());
+        
+        final InputStream in = dataPacket.getData();
+        StreamUtils.copy(in, encodedOut);
+        encodedOut.flush();
+    }
+
+    
+    @Override
+    public DataPacket decode(final InputStream stream) 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);
+        }
+        
+        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();
+        
+        return new StandardDataPacket(attributes, stream, numBytes);
+    }
+
+    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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/HandshakeException.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/PortNotRunningException.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
new file mode 100644
index 0000000..e12348a
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/ProtocolException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.io.IOException;
+
+public class ProtocolException extends IOException {
+
+    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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/exception/UnknownPortException.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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/05b64593/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java b/nifi/nifi-commons/nifi-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/nifi-commons/nifi-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();
+    }
+}


[15/29] incubator-nifi git commit: NIFI-282: Updated unit test

Posted by ma...@apache.org.
NIFI-282: Updated unit test


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/8f0402fb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/8f0402fb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/8f0402fb

Branch: refs/heads/develop
Commit: 8f0402fbbc75dc20ab226bb05f7cd38df6f929d6
Parents: 5c8a9e2
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Feb 12 19:41:51 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Feb 12 19:41:51 2015 -0500

----------------------------------------------------------------------
 .../client/socket/TestSiteToSiteClient.java     | 46 ++++++++++----------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8f0402fb/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
index 75becd3..8781421 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
@@ -36,7 +36,7 @@ import org.junit.Test;
 public class TestSiteToSiteClient {
 
     @Test
-    //@Ignore("For local testing only; not really a unit test but a manual test")
+    @Ignore("For local testing only; not really a unit test but a manual test")
     public void testReceive() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         
@@ -71,31 +71,31 @@ public class TestSiteToSiteClient {
     
     
     @Test
-    //@Ignore("For local testing only; not really a unit test but a manual test")
+    @Ignore("For local testing only; not really a unit test but a manual test")
     public void testSend() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         
-            final SiteToSiteClient client = new SiteToSiteClient.Builder()
-                .url("http://localhost:8080/nifi")
-                .portName("input")
-                .build();
-        
-            try {
-                final Transaction transaction = client.createTransaction(TransferDirection.SEND);
-                Assert.assertNotNull(transaction);
-                
-                    final Map<String, String> attrs = new HashMap<>();
-                    attrs.put("site-to-site", "yes, please!");
-                    final byte[] bytes = "Hello".getBytes();
-                    final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-                    final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
-                    transaction.send(packet);
-                
-                transaction.confirm();
-                transaction.complete();
-            } finally {
-                client.close();
-            }
+        final SiteToSiteClient client = new SiteToSiteClient.Builder()
+            .url("http://localhost:8080/nifi")
+            .portName("input")
+            .build();
+    
+        try {
+            final Transaction transaction = client.createTransaction(TransferDirection.SEND);
+            Assert.assertNotNull(transaction);
+            
+                final Map<String, String> attrs = new HashMap<>();
+                attrs.put("site-to-site", "yes, please!");
+                final byte[] bytes = "Hello".getBytes();
+                final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+                final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
+                transaction.send(packet);
+            
+            transaction.confirm();
+            transaction.complete();
+        } finally {
+            client.close();
+        }
     }
     
 }


[13/29] incubator-nifi git commit: Refactored client and add javadocs

Posted by ma...@apache.org.
Refactored client and add javadocs


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/d1e058cd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d1e058cd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d1e058cd

Branch: refs/heads/develop
Commit: d1e058cde7b011a4daa0d574d392569460fc70ba
Parents: 4ab5c30
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Feb 12 08:16:55 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Feb 12 08:16:55 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/remote/Communicant.java     |  47 +++
 .../main/java/org/apache/nifi/remote/Peer.java  |  19 +-
 .../nifi/remote/RemoteResourceInitiator.java    |   9 +
 .../org/apache/nifi/remote/Transaction.java     |  27 +-
 .../nifi/remote/TransactionCompletion.java      |  63 +++
 .../nifi/remote/client/SiteToSiteClient.java    |  31 +-
 .../remote/client/SiteToSiteClientConfig.java   |  21 +-
 .../client/socket/EndpointConnectionPool.java   | 113 ++++--
 .../nifi/remote/client/socket/SocketClient.java |  20 +-
 .../protocol/socket/SocketClientProtocol.java   |   7 +-
 .../socket/SocketClientTransaction.java         | 401 ++++++++++---------
 .../SocketClientTransactionCompletion.java      |  57 +++
 .../client/socket/TestSiteToSiteClient.java     |  16 +-
 .../nifi/remote/SocketRemoteSiteListener.java   |   8 +-
 .../nifi/remote/StandardRemoteGroupPort.java    | 263 +++++++-----
 15 files changed, 720 insertions(+), 382 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Communicant.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Communicant.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Communicant.java
new file mode 100644
index 0000000..ac2d498
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Communicant.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote;
+
+/**
+ * Represents the remote entity that the client is communicating with
+ */
+public interface Communicant {
+    /**
+     * Returns the NiFi site-to-site URL for the remote NiFi instance
+     * @return
+     */
+    String getUrl();
+    
+    /**
+     * The Host of the remote NiFi instance
+     * @return
+     */
+    String getHost();
+    
+    /**
+     * The Port that the remote NiFi instance is listening on for site-to-site communications
+     * @return
+     */
+    int getPort();
+    
+    /**
+     * The distinguished name that the remote NiFi instance has provided in its certificate if
+     * using secure communications, or <code>null</code> if the Distinguished Name is unknown
+     * @return
+     */
+    String getDistinguishedName();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
index dda5ae3..3534f95 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Peer.java
@@ -23,12 +23,13 @@ import java.util.Map;
 
 import org.apache.nifi.remote.protocol.CommunicationsSession;
 
-public class Peer {
+public class Peer implements Communicant {
 
     private final CommunicationsSession commsSession;
     private final String url;
     private final String clusterUrl;
     private final String host;
+    private final int port;
     
     private final Map<String, Long> penaltyExpirationMap = new HashMap<>();
     private boolean closed = false;
@@ -39,12 +40,15 @@ public class Peer {
         this.clusterUrl = clusterUrl;
 
         try {
-            this.host = new URI(peerUrl).getHost();
+            final URI uri = new URI(peerUrl);
+            this.port = uri.getPort();
+            this.host = uri.getHost();
         } catch (final Exception e) {
             throw new IllegalArgumentException("Invalid URL: " + peerUrl);
         }
     }
 
+    @Override
     public String getUrl() {
         return url;
     }
@@ -92,6 +96,7 @@ public class Peer {
         return closed;
     }
 
+    @Override
     public String getHost() {
         return host;
     }
@@ -127,4 +132,14 @@ public class Peer {
         sb.append("]");
         return sb.toString();
     }
+
+    @Override
+    public int getPort() {
+        return port;
+    }
+
+    @Override
+    public String getDistinguishedName() {
+        return commsSession.getUserDn();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
index 8eb5d8d..f469724 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/RemoteResourceInitiator.java
@@ -21,26 +21,33 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 
 import org.apache.nifi.remote.exception.HandshakeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class RemoteResourceInitiator {
 	public static final int RESOURCE_OK = 20;
 	public static final int DIFFERENT_RESOURCE_VERSION = 21;
 	public static final int ABORT = 255;
 
+	private static final Logger logger = LoggerFactory.getLogger(RemoteResourceInitiator.class);
 	
 	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
+	    logger.debug("Negotiating resource; proposal is {}", resource);
     	dos.writeUTF(resource.getResourceName());
     	final VersionNegotiator negotiator = resource.getVersionNegotiator();
     	dos.writeInt(negotiator.getVersion());
     	dos.flush();
         
         // wait for response from server.
+    	logger.debug("Receiving response from remote instance");
         final int statusCode = dis.read();
         switch (statusCode) {
             case RESOURCE_OK:	// server accepted our proposal of codec name/version
+                logger.debug("Response was RESOURCE_OK");
                 return resource;
             case DIFFERENT_RESOURCE_VERSION:	// server accepted our proposal of codec name but not the version
+                logger.debug("Response was DIFFERENT_RESOURCE_VERSION");
                 // Get server's preferred version
             	final int newVersion = dis.readInt();
                 
@@ -56,8 +63,10 @@ public class RemoteResourceInitiator {
                 // Attempt negotiation of resource based on our new preferred version.
                 return initiateResourceNegotiation(resource, dis, dos);
             case ABORT:
+                logger.debug("Response was ABORT");
             	throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF());
             default:
+                logger.debug("Response was {}; unable to negotiate codec", statusCode);
                 return null;	// Unable to negotiate codec
         }
 	}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
index 9fb6147..51bf244 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
@@ -121,30 +121,16 @@ public interface Transaction {
 	void confirm() throws IOException;
 	
 	/**
-	 * <p>
-	 * Completes the transaction and indicates to both the sender and receiver that the data transfer was
-	 * successful. If receiving data, this method can also optionally request that the sender back off sending
-	 * data for a short period of time. This is used, for instance, to apply backpressure or to notify the sender
-	 * that the receiver is not ready to receive data and made not service another request in the short term.
-	 * </p>
-	 * 
-	 * @param requestBackoff if <code>true</code> and the TransferDirection is RECEIVE, indicates to sender that it
-	 * should back off sending data for a short period of time. If <code>false</code> or if the TransferDirection of
-	 * this Transaction is SEND, then this argument is ignored.
-	 * 
-	 * @throws IOException
-	 */
-	void complete(boolean requestBackoff) throws IOException;
-	
-	/**
      * <p>
      * Completes the transaction and indicates to both the sender and receiver that the data transfer was
      * successful.
      * </p>
      * 
      * @throws IOException
+     * 
+     * @return a TransactionCompletion that contains details about the Transaction
      */
-	void complete() throws IOException;
+	TransactionCompletion complete() throws IOException;
 	
 	/**
 	 * <p>
@@ -174,6 +160,13 @@ public interface Transaction {
 	 */
 	TransactionState getState() throws IOException;
 	
+	/**
+	 * Returns a Communicant that represents the other side of this Transaction (i.e.,
+	 * the remote NiFi instance)
+	 * @return
+	 */
+	Communicant getCommunicant();
+	
 	
 	public enum TransactionState {
 	    /**

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransactionCompletion.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransactionCompletion.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransactionCompletion.java
new file mode 100644
index 0000000..be5f73a
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/TransactionCompletion.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.InputStream;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.remote.protocol.DataPacket;
+
+
+/**
+ * A TransactionCompletion provides information about a {@link Transaction} that has completed successfully.
+ */
+public interface TransactionCompletion {
+    
+    /**
+     * When a sending to a NiFi instance, the server may accept the content sent to it
+     * but indicate that its queues are full and that the client should backoff sending
+     * data for a bit. This method returns <code>true</code> if the server did in fact
+     * request that, <code>false</code> otherwise.
+     * @return
+     */
+    boolean isBackoff();
+    
+    /**
+     * Returns the number of Data Packets that were sent to or received from the remote
+     * NiFi instance in the Transaction
+     * @return
+     */
+    int getDataPacketsTransferred();
+    
+    /**
+     * Returns the number of bytes of DataPacket content that were sent to or received from 
+     * the remote NiFI instance in the Transaction. Note that this is different than the number
+     * of bytes actually transferred between the client and server, as it does not take into
+     * account the attributes or protocol-specific information that is exchanged but rather
+     * takes into account only the data in the {@link InputStream} of the {@link DataPacket}
+     * @return
+     */
+    long getBytesTransferred();
+    
+    /**
+     * Returns the amount of time that the Transaction took, from the time that the Transaction
+     * was created to the time that the Transaction was completed.
+     * @param timeUnit
+     * @return
+     */
+    long getDuration(TimeUnit timeUnit);
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
index 47568fd..0591b5a 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -126,6 +126,7 @@ public interface SiteToSiteClient extends Closeable {
 		private String url;
 		private long timeoutNanos = TimeUnit.SECONDS.toNanos(30);
 		private long penalizationNanos = TimeUnit.SECONDS.toNanos(3);
+		private long idleExpirationNanos = TimeUnit.SECONDS.toNanos(30L);
 		private SSLContext sslContext;
 		private EventReporter eventReporter;
 		private File peerPersistenceFile;
@@ -163,6 +164,19 @@ public interface SiteToSiteClient extends Closeable {
 		}
 		
 		/**
+		 * Specifies the amount of time that a connection can remain idle in the connection pool before it
+		 * is "expired" and shutdown. The default value is 30 seconds.
+		 *  
+		 * @param timeout
+		 * @param unit
+		 * @return
+		 */
+		public Builder idleExpiration(final long timeout, final TimeUnit unit) {
+		    this.idleExpirationNanos = unit.toNanos(timeout);
+		    return this;
+		}
+		
+		/**
 		 * If there is a problem communicating with a node (i.e., any node in the remote NiFi cluster
 		 * or the remote instance of NiFi if it is standalone), specifies how long the client should
 		 * wait before attempting to communicate with that node again. While a particular node is penalized,
@@ -327,6 +341,11 @@ public interface SiteToSiteClient extends Closeable {
 				}
 				
 				@Override
+				public long getIdleConnectionExpiration(final TimeUnit timeUnit) {
+				    return Builder.this.getIdleConnectionExpiration(timeUnit);
+				}
+				
+				@Override
 				public SSLContext getSslContext() {
 					return Builder.this.getSslContext();
 				}
@@ -384,12 +403,22 @@ public interface SiteToSiteClient extends Closeable {
 		}
 
 		/**
-		 * Returns the communications timeout in nanoseconds
+		 * Returns the communications timeout
 		 * @return
 		 */
 		public long getTimeout(final TimeUnit timeUnit) {
 			return timeUnit.convert(timeoutNanos, TimeUnit.NANOSECONDS);
 		}
+		
+		/**
+		 * Returns the amount of of time that a connection can remain idle in the connection
+		 * pool before being shutdown
+		 * @param timeUnit
+		 * @return
+		 */
+		public long getIdleConnectionExpiration(final TimeUnit timeUnit) {
+		    return timeUnit.convert(idleExpirationNanos, TimeUnit.NANOSECONDS);
+		}
 
 		/**
 		 * Returns the amount of time that a particular node will be ignored after a

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
index 37c48f8..d03ab3c 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
@@ -37,6 +37,14 @@ public interface SiteToSiteClientConfig {
 	 * @return
 	 */
 	long getTimeout(final TimeUnit timeUnit);
+	
+	/**
+	 * Returns the amount of time that a connection can remain idle before it is
+	 * "expired" and shut down
+	 * @param timeUnit
+	 * @return
+	 */
+	long getIdleConnectionExpiration(TimeUnit timeUnit);
 
 	/**
 	 * Returns the amount of time that a particular node will be ignored after a
@@ -53,12 +61,6 @@ public interface SiteToSiteClientConfig {
 	SSLContext getSslContext();
 	
 	/**
-	 * Returns the EventReporter that is to be used by clients to report events
-	 * @return
-	 */
-	EventReporter getEventReporter();
-
-	/**
 	 * Returns the file that is to be used for persisting the nodes of a remote cluster, if any.
 	 * @return
 	 */
@@ -111,4 +113,11 @@ public interface SiteToSiteClientConfig {
      * @return
      */
 	int getPreferredBatchCount();
+	
+	/**
+     * Returns the EventReporter that is to be used by clients to report events
+     * @return
+     */
+    EventReporter getEventReporter();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index 6869cca..43bc8e5 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -107,12 +107,13 @@ public class EndpointConnectionPool {
     private volatile List<PeerStatus> peerStatuses;
     private volatile long peerRefreshTime = 0L;
     private volatile PeerStatusCache peerStatusCache;
-    private final Set<CommunicationsSession> activeCommsChannels = new HashSet<>();
+    private final Set<EndpointConnection> activeConnections = Collections.synchronizedSet(new HashSet<EndpointConnection>());
 
     private final File peersFile;
     private final EventReporter eventReporter;
     private final SSLContext sslContext;
     private final ScheduledExecutorService taskExecutor;
+    private final int idleExpirationMillis;
     
     private final ReadWriteLock listeningPortRWLock = new ReentrantReadWriteLock();
     private final Lock remoteInfoReadLock = listeningPortRWLock.readLock();
@@ -124,12 +125,18 @@ public class EndpointConnectionPool {
     private final Map<String, String> outputPortMap = new HashMap<>();	// map output port name to identifier
     
     private volatile int commsTimeout;
-
-    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final EventReporter eventReporter, final File persistenceFile) {
-    	this(clusterUrl, commsTimeoutMillis, null, eventReporter, persistenceFile);
+    private volatile boolean shutdown = false;
+    
+    
+    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final int idleExpirationMillis, 
+            final EventReporter eventReporter, final File persistenceFile) 
+    {
+    	this(clusterUrl, commsTimeoutMillis, idleExpirationMillis, null, eventReporter, persistenceFile);
     }
     
-    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) {
+    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final int idleExpirationMillis,
+            final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) 
+    {
     	try {
     		this.clusterUrl = new URI(clusterUrl);
     	} catch (final URISyntaxException e) {
@@ -147,6 +154,7 @@ public class EndpointConnectionPool {
     	this.peersFile = persistenceFile;
     	this.eventReporter = eventReporter;
     	this.commsTimeout = commsTimeoutMillis;
+    	this.idleExpirationMillis = idleExpirationMillis;
     	
     	Set<PeerStatus> recoveredStatuses;
     	if ( persistenceFile != null && persistenceFile.exists() ) {
@@ -225,19 +233,21 @@ public class EndpointConnectionPool {
                 
                 // if we can't get an existing Connection, create one
                 if ( connection == null ) {
-                    logger.debug("No Connection available for Port {}; creating new Connection", remoteDestination.getIdentifier());
+                    logger.debug("{} No Connection available for Port {}; creating new Connection", this, remoteDestination.getIdentifier());
                     protocol = new SocketClientProtocol();
                     protocol.setDestination(remoteDestination);
 
+                    logger.debug("{} getting next peer status", this);
                     final PeerStatus peerStatus = getNextPeerStatus(direction);
+                    logger.debug("{} next peer status = {}", this, peerStatus);
                     if ( peerStatus == null ) {
                         return null;
                     }
 
                     try {
+                        logger.debug("{} Establishing site-to-site connection with {}", this, peerStatus);
                         commsSession = establishSiteToSiteConnection(peerStatus);
                     } catch (final IOException ioe) {
-                        // TODO: penalize peer status
                         penalize(peerStatus, remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS));
                         throw ioe;
                     }
@@ -245,6 +255,7 @@ public class EndpointConnectionPool {
                     final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
                     final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
                     try {
+                        logger.debug("{} Negotiating protocol", this);
                         RemoteResourceInitiator.initiateResourceNegotiation(protocol, dis, dos);
                     } catch (final HandshakeException e) {
                         try {
@@ -267,6 +278,7 @@ public class EndpointConnectionPool {
                     
                     // perform handshake
                     try {
+                        logger.debug("{} performing handshake", this);
                         protocol.handshake(peer);
                         
                         // handle error cases
@@ -286,7 +298,9 @@ public class EndpointConnectionPool {
                         }
                         
                         // negotiate the FlowFileCodec to use
+                        logger.debug("{} negotiating codec", this);
                         codec = protocol.negotiateCodec(peer);
+                        logger.debug("{} negotiated codec is {}", this, codec);
                     } catch (final PortNotRunningException | UnknownPortException e) {
                     	throw e;
                     } catch (final Exception e) {
@@ -323,6 +337,7 @@ public class EndpointConnectionPool {
             }
         }
         
+        activeConnections.add(connection);
         return connection;
     }
     
@@ -338,7 +353,14 @@ public class EndpointConnectionPool {
     		return false;
     	}
     	
-    	return connectionQueue.offer(endpointConnection);
+    	activeConnections.remove(endpointConnection);
+    	if ( shutdown ) {
+    	    terminate(endpointConnection);
+    	    return false;
+    	} else {
+    	    endpointConnection.setLastTimeUsed();
+    	    return connectionQueue.offer(endpointConnection);
+    	}
     }
     
     private void penalize(final PeerStatus status, final long penalizationMillis) {
@@ -393,27 +415,36 @@ public class EndpointConnectionPool {
         }
     }
     
+    private boolean isPeerRefreshNeeded(final List<PeerStatus> peerList) {
+        return (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD);
+    }
+    
     private PeerStatus getNextPeerStatus(final TransferDirection direction) {
         List<PeerStatus> peerList = peerStatuses;
-        if ( (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD) ) {
+        if ( isPeerRefreshNeeded(peerList) ) {
             peerRefreshLock.lock();
             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);
+                // now that we have the lock, check again that we need to refresh (because another thread
+                // could have been refreshing while we were waiting for the lock).
+                peerList = peerStatuses;
+                if (isPeerRefreshNeeded(peerList)) {
+                    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);
+                        }
+                        
+                        if ( eventReporter != null ) {
+                        	eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
+                        }
                     }
                     
-                    if ( eventReporter != null ) {
-                    	eventReporter.reportEvent(Severity.WARNING, CATEGORY, message);
-                    }
+                    this.peerStatuses = peerList;
+                    peerRefreshTime = System.currentTimeMillis();
                 }
-                
-                this.peerStatuses = peerList;
-                peerRefreshTime = System.currentTimeMillis();
             } finally {
                 peerRefreshLock.unlock();
             }
@@ -488,7 +519,10 @@ public class EndpointConnectionPool {
 
     private Set<PeerStatus> fetchRemotePeerStatuses() throws IOException, HandshakeException, UnknownPortException, PortNotRunningException {
     	final String hostname = clusterUrl.getHost();
-        final int port = getSiteToSitePort();
+        final Integer port = getSiteToSitePort();
+        if ( port == null ) {
+            throw new IOException("Remote instance of NiFi is not configured to allow site-to-site communications");
+        }
     	
     	final CommunicationsSession commsSession = establishSiteToSiteConnection(hostname, port);
         final Peer peer = new Peer(commsSession, "nifi://" + hostname + ":" + port, clusterUrl.toString());
@@ -667,7 +701,7 @@ public class EndpointConnectionPool {
         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");
+            distributionDescription.append("\n").append(entry.getKey()).append(" will receive ").append(percentage).append("% of data");
         }
         logger.info(distributionDescription.toString());
 
@@ -677,35 +711,36 @@ public class EndpointConnectionPool {
     
     
     private void cleanupExpiredSockets() {
-        final List<EndpointConnection> states = new ArrayList<>();
+        final List<EndpointConnection> connections = new ArrayList<>();
         
-        EndpointConnection state;
-        while ((state = connectionQueue.poll()) != null) {
+        EndpointConnection connection;
+        while ((connection = connectionQueue.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 ) {
+            final long lastUsed = connection.getLastTimeUsed();
+            if ( lastUsed < System.currentTimeMillis() - idleExpirationMillis ) {
                 try {
-                    state.getSocketClientProtocol().shutdown(state.getPeer());
+                    connection.getSocketClientProtocol().shutdown(connection.getPeer());
                 } catch (final Exception e) {
                     logger.debug("Failed to shut down {} using {} due to {}", 
-                        new Object[] {state.getSocketClientProtocol(), state.getPeer(), e} );
+                        new Object[] {connection.getSocketClientProtocol(), connection.getPeer(), e} );
                 }
                 
-                cleanup(state.getSocketClientProtocol(), state.getPeer());
+                terminate(connection);
             } else {
-                states.add(state);
+                connections.add(connection);
             }
         }
         
-        connectionQueue.addAll(states);
+        connectionQueue.addAll(connections);
     }
     
     public void shutdown() {
+        shutdown = true;
     	taskExecutor.shutdown();
     	peerTimeoutExpirations.clear();
-            
-        for ( final CommunicationsSession commsSession : activeCommsChannels ) {
-            commsSession.interrupt();
+        
+       for ( final EndpointConnection conn : activeConnections ) {
+           conn.getPeer().getCommunicationsSession().interrupt();
         }
         
         EndpointConnection state;
@@ -714,8 +749,8 @@ public class EndpointConnectionPool {
         }
     }
     
-    public void terminate(final EndpointConnection state) {
-        cleanup(state.getSocketClientProtocol(), state.getPeer());
+    public void terminate(final EndpointConnection connection) {
+        cleanup(connection.getSocketClientProtocol(), connection.getPeer());
     }
     
     private void refreshPeers() {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
index 6fa934b..aae19b3 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -19,8 +19,10 @@ package org.apache.nifi.remote.client.socket;
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.nifi.remote.Communicant;
 import org.apache.nifi.remote.RemoteDestination;
 import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransactionCompletion;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.SiteToSiteClient;
 import org.apache.nifi.remote.client.SiteToSiteClientConfig;
@@ -40,7 +42,8 @@ public class SocketClient implements SiteToSiteClient {
 	private volatile String portIdentifier;
 	
 	public SocketClient(final SiteToSiteClientConfig config) {
-		pool = new EndpointConnectionPool(config.getUrl(), (int) config.getTimeout(TimeUnit.MILLISECONDS), 
+		pool = new EndpointConnectionPool(config.getUrl(), (int) config.getTimeout(TimeUnit.MILLISECONDS),
+		        (int) config.getIdleConnectionExpiration(TimeUnit.MILLISECONDS),
 				config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile());
 		
 		this.config = config;
@@ -130,14 +133,9 @@ public class SocketClient implements SiteToSiteClient {
 			}
 
 			@Override
-			public void complete() throws IOException {
-			    complete(false);
-			}
-			
-			@Override
-			public void complete(final boolean requestBackoff) throws IOException {
+			public TransactionCompletion complete() throws IOException {
 				try {
-					transaction.complete(requestBackoff);
+					return transaction.complete();
 				} finally {
 				    final EndpointConnection state = connectionStateRef.get();
 				    if ( state != null ) {
@@ -187,7 +185,11 @@ public class SocketClient implements SiteToSiteClient {
 			public TransactionState getState() throws IOException {
 				return transaction.getState();
 			}
-			
+
+			@Override
+			public Communicant getCommunicant() {
+			    return transaction.getCommunicant();
+			}
 		};
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
index e321663..390f4fc 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@ -317,10 +317,7 @@ public class SocketClientProtocol implements ClientProtocol {
 		// Commit the session so that we have persisted the data
 		session.commit();
 
-		// We want to apply backpressure if the outgoing connections are full. I.e., there are no available relationships.
-		final boolean applyBackpressure = context.getAvailableRelationships().isEmpty();
-
-		transaction.complete(applyBackpressure);
+		transaction.complete();
 		logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
 
 		if ( !flowFilesReceived.isEmpty() ) {
@@ -397,7 +394,7 @@ public class SocketClientProtocol implements ClientProtocol {
 	        final String dataSize = FormatUtils.formatDataSize(bytesSent);
 	        
 	        session.commit();
-	        transaction.complete(false);
+	        transaction.complete();
 	        
 	        final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
 	        logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
index cf8f9b2..b2fffed 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
@@ -25,8 +25,10 @@ import java.util.zip.CRC32;
 import java.util.zip.CheckedInputStream;
 import java.util.zip.CheckedOutputStream;
 
+import org.apache.nifi.remote.Communicant;
 import org.apache.nifi.remote.Peer;
 import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransactionCompletion;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.codec.FlowFileCodec;
 import org.apache.nifi.remote.exception.ProtocolException;
@@ -40,7 +42,7 @@ import org.slf4j.LoggerFactory;
 public class SocketClientTransaction implements Transaction {
 	private static final Logger logger = LoggerFactory.getLogger(SocketClientTransaction.class);
 	
-	
+	private final long creationNanoTime = System.nanoTime();
 	private final CRC32 crc = new CRC32();
 	private final int protocolVersion;
 	private final FlowFileCodec codec;
@@ -54,6 +56,7 @@ public class SocketClientTransaction implements Transaction {
 	
 	private boolean dataAvailable = false;
 	private int transfers = 0;
+	private long contentBytes = 0;
 	private TransactionState state;
 	
 	SocketClientTransaction(final int protocolVersion, final String destinationId, final Peer peer, final FlowFileCodec codec, 
@@ -108,54 +111,59 @@ public class SocketClientTransaction implements Transaction {
 	@Override
 	public DataPacket receive() throws IOException {
 	    try {
-    		if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
-    			throw new IllegalStateException("Cannot receive data because Transaction State is " + state);
-    		}
-    		
-        	if ( direction == TransferDirection.SEND ) {
-        	    throw new IllegalStateException("Attempting to receive data but started a SEND Transaction");
-        	}
-        	
-        	// if we already know there's no data, just return null
-        	if ( !dataAvailable ) {
-        	    return null;
-        	}
-    
-        	// if we have already received a packet, check if another is available.
-        	if ( transfers > 0 ) {
-        	    // Determine if Peer will send us data or has no data to send us
-                final Response dataAvailableCode = Response.read(dis);
-                switch (dataAvailableCode.getCode()) {
-                    case CONTINUE_TRANSACTION:
-                        logger.debug("{} {} Indicates Transaction should continue", this, peer);
-                        this.dataAvailable = true;
-                        break;
-                    case FINISH_TRANSACTION:
-                        logger.debug("{} {} Indicates Transaction should finish", peer);
-                        this.dataAvailable = false;
-                        break;
-                    default:
-                        throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
+	        try {
+        		if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
+        			throw new IllegalStateException("Cannot receive data because Transaction State is " + state);
+        		}
+        		
+            	if ( direction == TransferDirection.SEND ) {
+            	    throw new IllegalStateException("Attempting to receive data but started a SEND Transaction");
+            	}
+            	
+            	// if we already know there's no data, just return null
+            	if ( !dataAvailable ) {
+            	    return null;
+            	}
+        
+            	// if we have already received a packet, check if another is available.
+            	if ( transfers > 0 ) {
+            	    // Determine if Peer will send us data or has no data to send us
+                    final Response dataAvailableCode = Response.read(dis);
+                    switch (dataAvailableCode.getCode()) {
+                        case CONTINUE_TRANSACTION:
+                            logger.debug("{} {} Indicates Transaction should continue", this, peer);
+                            this.dataAvailable = true;
+                            break;
+                        case FINISH_TRANSACTION:
+                            logger.debug("{} {} Indicates Transaction should finish", peer);
+                            this.dataAvailable = false;
+                            break;
+                        default:
+                            throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
+                    }
                 }
-            }
-        	
-        	// if no data available, return null
-        	if ( !dataAvailable ) {
-        	    return null;
-        	}
-        	
-            logger.debug("{} Receiving data from {}", this, peer);
-            final InputStream dataIn = compress ? new CompressionInputStream(dis) : dis;
-            final DataPacket packet = codec.decode(new CheckedInputStream(dataIn, crc));
-            
-            if ( packet == null ) {
-                this.dataAvailable = false;
-            } else {
-            	transfers++;
-            }
-            
-            this.state = TransactionState.DATA_EXCHANGED;
-            return packet;
+            	
+            	// if no data available, return null
+            	if ( !dataAvailable ) {
+            	    return null;
+            	}
+            	
+                logger.debug("{} Receiving data from {}", this, peer);
+                final InputStream dataIn = compress ? new CompressionInputStream(dis) : dis;
+                final DataPacket packet = codec.decode(new CheckedInputStream(dataIn, crc));
+                
+                if ( packet == null ) {
+                    this.dataAvailable = false;
+                } else {
+                	transfers++;
+                	contentBytes += packet.getSize();
+                }
+                
+                this.state = TransactionState.DATA_EXCHANGED;
+                return packet;
+	        } catch (final IOException ioe) {
+	            throw new IOException("Failed to receive data from " + peer + " due to " + ioe, ioe);
+	        }
 	    } catch (final Exception e) {
 	        error();
 	        throw e;
@@ -164,35 +172,40 @@ public class SocketClientTransaction implements Transaction {
 	
 	
 	@Override
-	public void send(DataPacket dataPacket) throws IOException {
+	public void send(final DataPacket dataPacket) throws IOException {
 	    try {
-    		if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
-    			throw new IllegalStateException("Cannot send data because Transaction State is " + state);
-    		}
-    
-            if ( direction == TransferDirection.RECEIVE ) {
-                throw new IllegalStateException("Attempting to send data but started a RECEIVE Transaction");
-            }
-    
-    		if ( transfers > 0 ) {
-                ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
-            }
-    
-            logger.debug("{} Sending data to {}", this, peer);
-    
-            final OutputStream dataOut = compress ? new CompressionOutputStream(dos) : dos;
-    		final OutputStream out = new CheckedOutputStream(dataOut, crc);
-            codec.encode(dataPacket, out);
-            
-            // 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 ( compress ) {
-            	out.close();
-            }
-            
-            transfers++;
-            this.state = TransactionState.DATA_EXCHANGED;
+	        try {
+        		if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) {
+        			throw new IllegalStateException("Cannot send data because Transaction State is " + state);
+        		}
+        
+                if ( direction == TransferDirection.RECEIVE ) {
+                    throw new IllegalStateException("Attempting to send data but started a RECEIVE Transaction");
+                }
+        
+        		if ( transfers > 0 ) {
+                    ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
+                }
+        
+                logger.debug("{} Sending data to {}", this, peer);
+        
+                final OutputStream dataOut = compress ? new CompressionOutputStream(dos) : dos;
+        		final OutputStream out = new CheckedOutputStream(dataOut, crc);
+                codec.encode(dataPacket, out);
+                
+                // 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 ( compress ) {
+                	out.close();
+                }
+                
+                transfers++;
+                contentBytes += dataPacket.getSize();
+                this.state = TransactionState.DATA_EXCHANGED;
+	        } catch (final IOException ioe) {
+	            throw new IOException("Failed to send data to " + peer + " due to " + ioe, ioe);
+	        }
 	    } catch (final Exception e) {
 	        error();
 	        throw e;
@@ -211,59 +224,56 @@ public class SocketClientTransaction implements Transaction {
 		    state = TransactionState.TRANSACTION_CANCELED;
 		} catch (final IOException ioe) {
 		    error();
-		    throw ioe;
+		    throw new IOException("Failed to send 'cancel transaction' message to " + peer + " due to " + ioe, ioe);
 		}
 	}
 	
-	@Override
-	public void complete() throws IOException {
-	    complete(false);
-	}
 	
 	@Override
-	public void complete(boolean requestBackoff) throws IOException {
+	public TransactionCompletion complete() throws IOException {
 	    try {
-    		if ( state != TransactionState.TRANSACTION_CONFIRMED ) {
-    			throw new IllegalStateException("Cannot complete transaction because state is " + state + 
-    					"; Transaction can only be completed when state is " + TransactionState.TRANSACTION_CONFIRMED);
-    		}
-    		
-    		if ( direction == TransferDirection.RECEIVE ) {
-    		    if ( transfers == 0 ) {
-    		        state = TransactionState.TRANSACTION_COMPLETED;
-    		        return;
-    		    }
-    		    
-                if ( requestBackoff ) {
-                    // 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 {
+	        try {
+        		if ( state != TransactionState.TRANSACTION_CONFIRMED ) {
+        			throw new IllegalStateException("Cannot complete transaction because state is " + state + 
+        					"; Transaction can only be completed when state is " + TransactionState.TRANSACTION_CONFIRMED);
+        		}
+        		
+        		boolean backoff = false;
+        		if ( direction == TransferDirection.RECEIVE ) {
+        		    if ( transfers == 0 ) {
+        		        state = TransactionState.TRANSACTION_COMPLETED;
+        		        return new SocketClientTransactionCompletion(false, 0, 0L, System.nanoTime() - creationNanoTime);
+        		    }
+        		    
                     // 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);
+                    
+                    state = TransactionState.TRANSACTION_COMPLETED;
+                } else {
+                    final Response transactionResponse;
+                    try {
+                        transactionResponse = Response.read(dis);
+                    } catch (final IOException e) {
+                        throw new IOException(this + " Failed to receive a response from " + peer + " when expecting a TransactionFinished Indicator. " +
+                                "It is unknown whether or not the peer successfully received/processed the data.", e);
+                    }
+                    
+                    logger.debug("{} Received {} from {}", this, transactionResponse, peer);
+                    if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
+                        peer.penalize(destinationId, penaltyMillis);
+                        backoff = true;
+                    } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
+                        throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
+                    }
+                    
+                    state = TransactionState.TRANSACTION_COMPLETED;
                 }
-                
-                state = TransactionState.TRANSACTION_COMPLETED;
-            } else {
-                final Response transactionResponse;
-                try {
-                    transactionResponse = Response.read(dis);
-                } catch (final IOException e) {
-                    throw new IOException(this + " Failed to receive a response from " + peer + " when expecting a TransactionFinished Indicator. " +
-                            "It is unknown whether or not the peer successfully received/processed the data.", e);
-                }
-                
-                logger.debug("{} Received {} from {}", this, transactionResponse, peer);
-                if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
-                    peer.penalize(destinationId, penaltyMillis);
-                } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
-                    throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
-                }
-                
-                state = TransactionState.TRANSACTION_COMPLETED;
-            }
+        		
+        		return new SocketClientTransactionCompletion(backoff, transfers, contentBytes, System.nanoTime() - creationNanoTime);
+	        } catch (final IOException ioe) {
+	            throw new IOException("Failed to complete transaction with " + peer + " due to " + ioe, ioe);
+	        }
 	    } catch (final Exception e) {
 	        error();
 	        throw e;
@@ -274,81 +284,85 @@ public class SocketClientTransaction implements Transaction {
 	@Override
 	public void confirm() throws IOException {
 	    try {
-	        if ( state == TransactionState.TRANSACTION_STARTED && !dataAvailable && direction == TransferDirection.RECEIVE ) {
-	            // client requested to receive data but no data available. no need to confirm.
-	            state = TransactionState.TRANSACTION_CONFIRMED;
-	            return;
-	        }
-	        
-    		if ( state != TransactionState.DATA_EXCHANGED ) {
-    			throw new IllegalStateException("Cannot confirm Transaction because state is " + state + 
-    					"; Transaction can only be confirmed when state is " + TransactionState.DATA_EXCHANGED );
-    		}
-    
-            if ( direction == TransferDirection.RECEIVE ) {
-                if ( dataAvailable ) {
-                    throw new IllegalStateException("Cannot complete transaction because the sender has already sent more data than client has consumed.");
-                }
-                
-                // 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);
-                final String calculatedCRC = String.valueOf(crc.getValue());
-                ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
-                
-                final Response confirmTransactionResponse;
-                try {
-                    confirmTransactionResponse = Response.read(dis);
-                } catch (final IOException ioe) {
-                    logger.error("Failed to receive response code from {} when expected confirmation of transaction", peer);
-                    throw ioe;
-                }
-                
-                logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
-                
-                switch (confirmTransactionResponse.getCode()) {
-                    case CONFIRM_TRANSACTION:
-                        break;
-                    case BAD_CHECKSUM:
-                        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");
-                }
-                
-                state = TransactionState.TRANSACTION_CONFIRMED;
-            } else {
-                logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
-                ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
-                
-                final String calculatedCRC = String.valueOf(crc.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();
+	        try {
+    	        if ( state == TransactionState.TRANSACTION_STARTED && !dataAvailable && direction == TransferDirection.RECEIVE ) {
+    	            // client requested to receive data but no data available. no need to confirm.
+    	            state = TransactionState.TRANSACTION_CONFIRMED;
+    	            return;
+    	        }
+    	        
+        		if ( state != TransactionState.DATA_EXCHANGED ) {
+        			throw new IllegalStateException("Cannot confirm Transaction because state is " + state + 
+        					"; Transaction can only be confirmed when state is " + TransactionState.DATA_EXCHANGED );
+        		}
+        
+                if ( direction == TransferDirection.RECEIVE ) {
+                    if ( dataAvailable ) {
+                        throw new IllegalStateException("Cannot complete transaction because the sender has already sent more data than client has consumed.");
+                    }
                     
-                    // CRC was not used before version 4
-                    if ( protocolVersion > 3 ) {
-                        if ( !receivedCRC.equals(calculatedCRC) ) {
-                            ResponseCode.BAD_CHECKSUM.writeResponse(dos);
-                            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");
-                        }
+                    // 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);
+                    final String calculatedCRC = String.valueOf(crc.getValue());
+                    ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
+                    
+                    final Response confirmTransactionResponse;
+                    try {
+                        confirmTransactionResponse = Response.read(dis);
+                    } catch (final IOException ioe) {
+                        logger.error("Failed to receive response code from {} when expected confirmation of transaction", peer);
+                        throw ioe;
                     }
                     
-                    ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
+                    logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
+                    
+                    switch (confirmTransactionResponse.getCode()) {
+                        case CONFIRM_TRANSACTION:
+                            break;
+                        case BAD_CHECKSUM:
+                            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");
+                    }
+                    
+                    state = TransactionState.TRANSACTION_CONFIRMED;
                 } else {
-                    throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
+                    logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
+                    ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
+                    
+                    final String calculatedCRC = String.valueOf(crc.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();
+                        
+                        // CRC was not used before version 4
+                        if ( protocolVersion > 3 ) {
+                            if ( !receivedCRC.equals(calculatedCRC) ) {
+                                ResponseCode.BAD_CHECKSUM.writeResponse(dos);
+                                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);
+                    }
+                    
+                    state = TransactionState.TRANSACTION_CONFIRMED;
                 }
-                
-                state = TransactionState.TRANSACTION_CONFIRMED;
-            }
+	        } catch (final IOException ioe) {
+	            throw new IOException("Failed to confirm transaction with " + peer + " due to " + ioe, ioe);
+	        }
 	    } catch (final Exception e) {
 	        error();
 	        throw e;
@@ -365,4 +379,13 @@ public class SocketClientTransaction implements Transaction {
 		return state;
 	}
 
+	@Override
+	public Communicant getCommunicant() {
+	    return peer;
+	}
+	
+    @Override
+    public String toString() {
+        return "SocketClientTransaction[Url=" + peer.getUrl() + ", TransferDirection=" + direction + ", State=" + state + "]"; 
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransactionCompletion.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransactionCompletion.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransactionCompletion.java
new file mode 100644
index 0000000..5eb6c91
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransactionCompletion.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.concurrent.TimeUnit;
+
+import org.apache.nifi.remote.TransactionCompletion;
+
+public class SocketClientTransactionCompletion implements TransactionCompletion {
+
+    private final boolean backoff;
+    private final int dataPacketsTransferred;
+    private final long bytesTransferred;
+    private final long durationNanos;
+    
+    public SocketClientTransactionCompletion(final boolean backoff, final int dataPacketsTransferred, final long bytesTransferred, final long durationNanos) {
+        this.backoff = backoff;
+        this.dataPacketsTransferred = dataPacketsTransferred;
+        this.bytesTransferred = bytesTransferred;
+        this.durationNanos = durationNanos;
+    }
+
+    @Override
+    public boolean isBackoff() {
+        return backoff;
+    }
+
+    @Override
+    public int getDataPacketsTransferred() {
+        return dataPacketsTransferred;
+    }
+
+    @Override
+    public long getBytesTransferred() {
+        return bytesTransferred;
+    }
+
+    @Override
+    public long getDuration(final TimeUnit timeUnit) {
+        return timeUnit.convert(durationNanos, TimeUnit.NANOSECONDS);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
index a744905..2fd90f8 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransferDirection;
@@ -35,13 +36,13 @@ import org.junit.Test;
 public class TestSiteToSiteClient {
 
     @Test
-    @Ignore("For local testing only; not really a unit test but a manual test")
+    //@Ignore("For local testing only; not really a unit test but a manual test")
     public void testReceive() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         
         final SiteToSiteClient client = new SiteToSiteClient.Builder()
             .url("http://localhost:8080/nifi")
-            .portName("out")
+            .portName("cba")
             .requestBatchCount(1)
             .build();
         
@@ -62,7 +63,7 @@ public class TestSiteToSiteClient {
             Assert.assertNull(transaction.receive());
             
             transaction.confirm();
-            transaction.complete(false);
+            transaction.complete();
         } finally {
             client.close();
         }
@@ -70,13 +71,14 @@ public class TestSiteToSiteClient {
     
     
     @Test
-    @Ignore("For local testing only; not really a unit test but a manual test")
+    //@Ignore("For local testing only; not really a unit test but a manual test")
     public void testSend() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         
         final SiteToSiteClient client = new SiteToSiteClient.Builder()
-            .url("http://localhost:8080/nifi")
-            .portName("in")
+            .url("http://10.0.64.63:8080/nifi")
+            .portName("input")
+            .nodePenalizationPeriod(10, TimeUnit.MILLISECONDS)
             .build();
         
         try {
@@ -91,7 +93,7 @@ public class TestSiteToSiteClient {
             transaction.send(packet);
 
             transaction.confirm();
-            transaction.complete(false);
+            transaction.complete();
         } finally {
             client.close();
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d1e058cd/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
index 3295956..8a4839b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
@@ -22,6 +22,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
@@ -130,7 +131,9 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
                     final Thread thread = new Thread(new Runnable() {
                         @Override
                         public void run() {
-                            String hostname = socket.getInetAddress().getHostName();
+                            LOG.debug("{} Determining URL of connection", this);
+                            final InetAddress inetAddress = socket.getInetAddress();
+                            String hostname = inetAddress.getHostName();
                             final int slashIndex = hostname.indexOf("/");
                             if ( slashIndex == 0 ) {
                                 hostname = hostname.substring(1);
@@ -140,6 +143,7 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
 
                             final int port = socket.getPort();
                             final String peerUri = "nifi://" + hostname + ":" + port;
+                            LOG.debug("{} Connection URL is {}", this, peerUri);
                             
                             final CommunicationsSession commsSession;
                             final String dn;
@@ -154,6 +158,7 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
                                     dn = sslSocketChannel.getDn();
                                     commsSession.setUserDn(dn);
                                 } else {
+                                    LOG.trace("{} Channel is not secure", this);
                                     commsSession = new SocketChannelCommunicationsSession(socketChannel, peerUri);
                                     dn = null;
                                 }
@@ -306,6 +311,7 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
                         }
                     });
                     thread.setName("Site-to-Site Worker Thread-" + (threadCount++));
+                    LOG.debug("Handing connection to {}", thread);
                     thread.start();
                 }
             }


[19/29] incubator-nifi git commit: NIFI-282: Code cleanup

Posted by ma...@apache.org.
NIFI-282: Code cleanup


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/46eebc77
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/46eebc77
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/46eebc77

Branch: refs/heads/develop
Commit: 46eebc77abb726ecc56ae7fdc71d575a1f23f39c
Parents: a7405b9
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 16 16:53:38 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 16 16:53:38 2015 -0500

----------------------------------------------------------------------
 .../nifi/remote/client/socket/EndpointConnectionPool.java       | 5 ++++-
 .../nifi/remote/protocol/socket/SocketClientTransaction.java    | 2 +-
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/46eebc77/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index 450daec..42428f6 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -904,7 +904,10 @@ public class EndpointConnectionPool {
         return listeningPort;
     }
  
-    
+    @Override
+    public String toString() {
+        return "EndpointConnectionPool[Cluster URL=" + clusterUrl + "]";
+    }
     
     
     /**

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/46eebc77/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
index 2fbcfc4..a1ce07e 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
@@ -138,7 +138,7 @@ public class SocketClientTransaction implements Transaction {
                             this.dataAvailable = true;
                             break;
                         case FINISH_TRANSACTION:
-                            logger.debug("{} {} Indicates Transaction should finish", peer);
+                            logger.debug("{} {} Indicates Transaction should finish", this, peer);
                             this.dataAvailable = false;
                             break;
                         default:


[22/29] incubator-nifi git commit: NIFI-282: - Fixing pom parent and artifact version.

Posted by ma...@apache.org.
NIFI-282:
- Fixing pom parent and artifact version.

Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/dbae98f2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/dbae98f2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/dbae98f2

Branch: refs/heads/develop
Commit: dbae98f221fd97cc92f8e03c4c57a392d5c9b272
Parents: 8506a0c
Author: Matt Gilman <ma...@gmail.com>
Authored: Thu Feb 19 11:09:17 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Feb 19 11:09:17 2015 -0500

----------------------------------------------------------------------
 nifi/nifi-external/nifi-spark-receiver/pom.xml | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/dbae98f2/nifi/nifi-external/nifi-spark-receiver/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-external/nifi-spark-receiver/pom.xml b/nifi/nifi-external/nifi-spark-receiver/pom.xml
index b21d554..75d71f2 100644
--- a/nifi/nifi-external/nifi-spark-receiver/pom.xml
+++ b/nifi/nifi-external/nifi-spark-receiver/pom.xml
@@ -17,7 +17,7 @@
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi</artifactId>
+        <artifactId>nifi-external</artifactId>
         <version>0.0.2-incubating-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.nifi</groupId>
@@ -32,7 +32,6 @@
 		<dependency>
 			<groupId>org.apache.nifi</groupId>
 			<artifactId>nifi-site-to-site-client</artifactId>
-			<version>0.0.2-incubating-SNAPSHOT</version>
 		</dependency>
 	</dependencies>
 </project>
\ No newline at end of file


[07/29] incubator-nifi git commit: NIFI-282: Refactoring to allow for separate client

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index a51cdba..1e33e1f 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.remote;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -23,6 +24,7 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -37,10 +39,9 @@ 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.client.socket.EndpointConnection;
+import org.apache.nifi.remote.client.socket.EndpointConnectionPool;
 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.TransmissionDisabledException;
@@ -50,6 +51,7 @@ 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;
 
@@ -66,9 +68,10 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     private final AtomicBoolean useCompression = new AtomicBoolean(false);
     private final AtomicBoolean targetExists = new AtomicBoolean(true);
     private final AtomicBoolean targetRunning = new AtomicBoolean(true);
+    private final SSLContext sslContext;
     private final TransferDirection transferDirection;
     
-    private final EndpointConnectionStatePool connectionStatePool;
+    private final AtomicReference<EndpointConnectionPool> connectionPoolRef = new AtomicReference<>();
     
     private final Set<CommunicationsSession> activeCommsChannels = new HashSet<>();
     private final Lock interruptLock = new ReentrantLock();
@@ -83,9 +86,13 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         
         this.remoteGroup = remoteGroup;
         this.transferDirection = direction;
+        this.sslContext = sslContext;
         setScheduldingPeriod(MINIMUM_SCHEDULING_NANOS + " nanos");
-        
-        connectionStatePool = remoteGroup.getConnectionPool();
+    }
+    
+    private static File getPeerPersistenceFile(final String portId) {
+        final File stateDir = NiFiProperties.getInstance().getPersistentStateDirectory();
+        return new File(stateDir, portId + ".peers");
     }
     
     @Override
@@ -111,6 +118,11 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         } finally {
             interruptLock.unlock();
         }
+        
+        final EndpointConnectionPool pool = connectionPoolRef.get();
+        if ( pool != null ) {
+            pool.shutdown();
+        }
     }
     
     @Override
@@ -123,6 +135,11 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         } finally {
             interruptLock.unlock();
         }
+        
+        final EndpointConnectionPool connectionPool = new EndpointConnectionPool(remoteGroup.getTargetUri().toString(), 
+                remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS), 
+                sslContext, remoteGroup.getEventReporter(), getPeerPersistenceFile(getIdentifier()));
+        connectionPoolRef.set(connectionPool);
     }
     
     
@@ -140,9 +157,10 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         
         String url = getRemoteProcessGroup().getTargetUri().toString();
         
-        final EndpointConnectionState connectionState;
+        final EndpointConnectionPool connectionPool = connectionPoolRef.get();
+        final EndpointConnection connection;
         try {
-        	connectionState = connectionStatePool.getEndpointConnectionState(this, transferDirection);
+        	connection = connectionPool.getEndpointConnection(this, transferDirection);
         } catch (final PortNotRunningException e) {
             context.yield();
             this.targetRunning.set(false);
@@ -157,7 +175,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             logger.error(message);
             remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
             return;
-        } catch (final HandshakeException | IOException e) {
+        } catch (final 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() ) {
@@ -168,15 +186,15 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             return;
         }
         
-        if ( connectionState == null ) {
+        if ( connection == null ) {
             logger.debug("{} Unable to determine the next peer to communicate with; all peers must be penalized, so yielding context", this);
             context.yield();
             return;
         }
         
-        FlowFileCodec codec = connectionState.getCodec();
-        SocketClientProtocol protocol = connectionState.getSocketClientProtocol();
-        final Peer peer = connectionState.getPeer();
+        FlowFileCodec codec = connection.getCodec();
+        SocketClientProtocol protocol = connection.getSocketClientProtocol();
+        final Peer peer = connection.getPeer();
         url = peer.getUrl();
         
         try {
@@ -194,7 +212,10 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             if ( getConnectableType() == ConnectableType.REMOTE_INPUT_PORT ) {
                 transferFlowFiles(peer, protocol, context, session, codec);
             } else {
-                receiveFlowFiles(peer, protocol, context, session, codec);
+                final int numReceived = receiveFlowFiles(peer, protocol, context, session, codec);
+                if ( numReceived == 0 ) {
+                    context.yield();
+                }
             }
 
             interruptLock.lock();
@@ -210,13 +231,13 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
 
             session.commit();
             
-            connectionState.setLastTimeUsed();
-            connectionStatePool.offer(connectionState);
+            connection.setLastTimeUsed();
+            connectionPool.offer(connection);
         } catch (final TransmissionDisabledException e) {
             cleanup(protocol, peer);
             session.rollback();
         } catch (final Exception e) {
-            connectionStatePool.penalize(peer, getYieldPeriod(TimeUnit.MILLISECONDS));
+            connectionPool.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);
@@ -261,12 +282,12 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     }
     
     
-    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);
+    private int transferFlowFiles(final Peer peer, final ClientProtocol protocol, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+        return protocol.transferFlowFiles(peer, context, session, codec);
     }
     
-    private void receiveFlowFiles(final Peer peer, final ClientProtocol protocol, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
-        protocol.receiveFlowFiles(peer, context, session, codec);
+    private int receiveFlowFiles(final Peer peer, final ClientProtocol protocol, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+        return protocol.receiveFlowFiles(peer, context, session, codec);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.java
deleted file mode 100644
index dca1d84..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.java
deleted file mode 100644
index 60ef33f..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.java
deleted file mode 100644
index dc3d68f..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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/081471c4/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index eb22b0e..63c960d 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@ -430,7 +430,7 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
         
         logger.debug("{} received {} from {}", new Object[] {this, transactionResponse, peer});
         if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
-            peer.penalize(port.getYieldPeriod(TimeUnit.MILLISECONDS));
+            peer.penalize(port.getIdentifier(), port.getYieldPeriod(TimeUnit.MILLISECONDS));
         } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
             throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
         }


[10/29] incubator-nifi git commit: Merge branch 'develop' into nifi-site-to-site-client

Posted by ma...@apache.org.
Merge branch 'develop' into nifi-site-to-site-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/4c04fe65
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/4c04fe65
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/4c04fe65

Branch: refs/heads/develop
Commit: 4c04fe65285ca90d86e937eae3438f4dddb6c86f
Parents: 081471c 0133f84
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 9 20:42:37 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 9 20:42:37 2015 -0500

----------------------------------------------------------------------
 nifi-site/Gruntfile.js                          |   5 +-
 nifi-site/src/includes/topbar.hbs               |   6 +-
 nifi-site/src/js/app.js                         |   2 +
 nifi-site/src/pages/html/developer-guide.hbs    |   7 +
 nifi-site/src/pages/html/download.hbs           |   2 +-
 nifi-site/src/pages/html/overview.hbs           |   7 +
 nifi-site/src/pages/html/screencasts.hbs        |  15 +
 nifi-site/src/pages/html/user-guide.hbs         |   7 +
 nifi-site/src/scss/app.scss                     |  22 +
 nifi/LICENSE                                    | 343 +++-----
 nifi/NOTICE                                     |   8 -
 nifi/README.md                                  |  26 +-
 nifi/nifi-assembly/LICENSE                      | 803 +++++++++++++++++++
 nifi/nifi-assembly/NOTICE                       | 529 ++++++++++++
 nifi/nifi-assembly/README.md                    | 124 +++
 nifi/nifi-assembly/pom.xml                      |   6 +-
 .../src/main/assembly/dependencies.xml          |   6 +-
 nifi/nifi-docs/LICENSE                          | 235 ++++++
 nifi/nifi-docs/NOTICE                           |   5 +
 nifi/nifi-docs/pom.xml                          |  30 +-
 .../src/main/asciidoc/asciidoc-mod.css          | 418 ++++++++++
 .../src/main/asciidoc/developer-guide.adoc      |  32 +-
 .../src/main/assembly/dependencies.xml          |  16 +
 .../src/main/resources/META-INF/LICENSE         | 264 ++++++
 .../src/main/resources/META-INF/NOTICE          | 131 +++
 .../nifi-web/nifi-web-api/pom.xml               |   2 +-
 .../src/main/resources/META-INF/NOTICE          |  27 +
 .../src/main/resources/META-INF/NOTICE          |  19 +
 .../src/main/resources/META-INF/LICENSE         | 434 ++++++++++
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    |   2 +-
 .../src/main/resources/META-INF/NOTICE          |  35 +
 .../src/main/resources/META-INF/LICENSE         | 359 +++++++++
 .../src/main/resources/META-INF/NOTICE          | 237 ++++++
 .../src/main/resources/META-INF/NOTICE          |  37 +
 .../src/main/resources/META-INF/LICENSE         | 299 +++++++
 .../src/main/resources/META-INF/NOTICE          |  72 ++
 .../src/main/resources/META-INF/NOTICE          | 202 +++++
 .../src/main/resources/META-INF/LICENSE         | 292 +++++++
 .../src/main/resources/META-INF/NOTICE          | 119 +++
 .../src/main/resources/META-INF/NOTICE          |  19 +
 .../src/main/resources/META-INF/NOTICE          |  19 +
 .../src/main/resources/META-INF/NOTICE          |  19 +
 .../src/main/resources/META-INF/LICENSE         | 238 ++++++
 .../src/main/resources/META-INF/NOTICE          |  49 ++
 nifi/pom.xml                                    |   5 +-
 45 files changed, 5241 insertions(+), 293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4c04fe65/nifi/pom.xml
----------------------------------------------------------------------


[17/29] incubator-nifi git commit: NIFI-282: Added send(byte[], Map) method to avoid having to create a DataPacket object

Posted by ma...@apache.org.
NIFI-282: Added send(byte[], Map<String, String>) method to avoid having to create a DataPacket object


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/2f60ddc0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/2f60ddc0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/2f60ddc0

Branch: refs/heads/develop
Commit: 2f60ddc03a3e867ea3b0826621aa63439a10bee7
Parents: e16fc79
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 16 15:18:57 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 16 15:18:57 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/remote/Transaction.java  | 11 +++++++++++
 .../nifi/remote/client/socket/SocketClient.java   |  6 ++++++
 .../apache/nifi/remote/protocol/DataPacket.java   | 18 +++++++++++++++++-
 .../protocol/socket/SocketClientTransaction.java  |  8 ++++++++
 4 files changed, 42 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2f60ddc0/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
index 51bf244..eb7312d 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/Transaction.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.remote;
 
 import java.io.IOException;
+import java.util.Map;
 
 import org.apache.nifi.remote.protocol.DataPacket;
 
@@ -81,6 +82,16 @@ public interface Transaction {
     void send(DataPacket dataPacket) throws IOException;
     
     /**
+     * Sends the given byte array as the content of a {@link DataPacket} along with the
+     * provided attributes
+     * 
+     * @param content
+     * @param attributes
+     * @throws IOException
+     */
+    void send(byte[] content, Map<String, String> attributes) throws IOException;
+    
+    /**
      * Retrieves information from the remote NiFi instance, if any is available. If no data is available, will return
      * {@code null}. It is important to consume all data from the remote NiFi instance before attempting to 
      * call {@link #confirm()}. This is because the sender is always responsible for determining when the Transaction

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2f60ddc0/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
index c11c2ab..bd9319f 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.remote.client.socket;
 
 import java.io.IOException;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.remote.Communicant;
@@ -186,6 +187,11 @@ public class SocketClient implements SiteToSiteClient {
 			}
 
 			@Override
+			public void send(final byte[] content, final Map<String, String> attributes) throws IOException {
+			    transaction.send(content, attributes);
+			}
+			
+			@Override
 			public DataPacket receive() throws IOException {
 				return transaction.receive();
 			}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2f60ddc0/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java
index f4fa4d0..3f0ec4f 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/DataPacket.java
@@ -19,11 +19,27 @@ package org.apache.nifi.remote.protocol;
 import java.io.InputStream;
 import java.util.Map;
 
+
+/**
+ * Represents a piece of data that is to be sent to or that was received from a NiFi instance.
+ */
 public interface DataPacket {
 
+    /**
+     * The key-value attributes that are to be associated with the data
+     * @return
+     */
 	Map<String, String> getAttributes();
 	
+	/**
+	 * An InputStream from which the content can be read
+	 * @return
+	 */
 	InputStream getData();
-	
+
+	/**
+	 * The length of the InputStream.
+	 * @return
+	 */
 	long getSize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2f60ddc0/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
index b2fffed..2fbcfc4 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java
@@ -16,11 +16,13 @@
  */
 package org.apache.nifi.remote.protocol.socket;
 
+import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Map;
 import java.util.zip.CRC32;
 import java.util.zip.CheckedInputStream;
 import java.util.zip.CheckedOutputStream;
@@ -36,6 +38,7 @@ import org.apache.nifi.remote.io.CompressionInputStream;
 import org.apache.nifi.remote.io.CompressionOutputStream;
 import org.apache.nifi.remote.protocol.DataPacket;
 import org.apache.nifi.remote.protocol.RequestType;
+import org.apache.nifi.remote.util.StandardDataPacket;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -172,6 +175,11 @@ public class SocketClientTransaction implements Transaction {
 	
 	
 	@Override
+	public void send(final byte[] content, final Map<String, String> attributes) throws IOException {
+	    send(new StandardDataPacket(attributes, new ByteArrayInputStream(content), content.length));
+	}
+	
+	@Override
 	public void send(final DataPacket dataPacket) throws IOException {
 	    try {
 	        try {


[23/29] incubator-nifi git commit: NIFI-362: Avoid continually scheduling components to run if there is no work for them to do or if they are yielded

Posted by ma...@apache.org.
NIFI-362: Avoid continually scheduling components to run if there is no work for them to do or if they are yielded


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/4cc106a5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/4cc106a5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/4cc106a5

Branch: refs/heads/develop
Commit: 4cc106a54d9b6528e38cb99ecb15524a07a1f0c9
Parents: dde5fd5
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Feb 22 10:53:24 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Feb 22 10:53:24 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/controller/StandardFunnel.java  |   2 +-
 .../scheduling/QuartzSchedulingAgent.java       |  21 +++-
 .../controller/scheduling/ScheduleState.java    |  18 ++--
 .../scheduling/TimerDrivenSchedulingAgent.java  | 105 ++++++++++++++++---
 .../tasks/ContinuallyRunConnectableTask.java    |  32 ++++--
 .../tasks/ContinuallyRunProcessorTask.java      |  32 +++---
 6 files changed, 163 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4cc106a5/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
index e34e043..3bdfd20 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
@@ -94,7 +94,7 @@ public class StandardFunnel implements Funnel {
         position = new AtomicReference<>(new Position(0D, 0D));
         scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
         penalizationPeriod = new AtomicReference<>("30 sec");
-        yieldPeriod = new AtomicReference<>("1 sec");
+        yieldPeriod = new AtomicReference<>("250 millis");
         yieldExpiration = new AtomicLong(0L);
         schedulingPeriod = new AtomicReference<>("0 millis");
         schedulingNanos = new AtomicLong(30000);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4cc106a5/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
index ea67492..3355e73 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
@@ -21,6 +21,7 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -34,8 +35,9 @@ import org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask;
 import org.apache.nifi.controller.tasks.ReportingTaskWrapper;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.engine.FlowEngine;
+import org.apache.nifi.processor.StandardProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.FormatUtils;
-
 import org.quartz.CronExpression;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -130,13 +132,16 @@ public class QuartzSchedulingAgent implements SchedulingAgent {
 
         final List<AtomicBoolean> triggers = new ArrayList<>();
         for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) {
-            final Runnable continuallyRunTask;
+            final Callable<Boolean> continuallyRunTask;
             if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
                 final ProcessorNode procNode = (ProcessorNode) connectable;
-                ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, encryptor);
+                
+                final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor);
+                ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, standardProcContext);
                 continuallyRunTask = runnableTask;
             } else {
-                continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, encryptor);
+                final ConnectableProcessContext connProcContext = new ConnectableProcessContext(connectable, encryptor);
+                continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, connProcContext);
             }
 
             final AtomicBoolean canceled = new AtomicBoolean(false);
@@ -147,7 +152,13 @@ public class QuartzSchedulingAgent implements SchedulingAgent {
                         return;
                     }
 
-                    continuallyRunTask.run();
+                    try {
+                        continuallyRunTask.call();
+                    } catch (final RuntimeException re) {
+                        throw re;
+                    } catch (final Exception e) {
+                        throw new ProcessException(e);
+                    }
 
                     if (canceled.get()) {
                         return;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4cc106a5/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
index eb5a437..ff17912 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
@@ -16,9 +16,10 @@
  */
 package org.apache.nifi.controller.scheduling;
 
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
-import java.util.List;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -27,7 +28,7 @@ public class ScheduleState {
 
     private final AtomicInteger activeThreadCount = new AtomicInteger(0);
     private final AtomicBoolean scheduled = new AtomicBoolean(false);
-    private final List<ScheduledFuture<?>> futures = new ArrayList<>();
+    private final Set<ScheduledFuture<?>> futures = new HashSet<ScheduledFuture<?>>();
     private final AtomicBoolean mustCallOnStoppedMethods = new AtomicBoolean(false);
     private volatile long lastStopTime = -1;
 
@@ -79,12 +80,17 @@ public class ScheduleState {
      *
      * @param newFutures
      */
-    public void setFutures(final List<ScheduledFuture<?>> newFutures) {
+    public synchronized void setFutures(final Collection<ScheduledFuture<?>> newFutures) {
         futures.clear();
         futures.addAll(newFutures);
     }
 
-    public List<ScheduledFuture<?>> getFutures() {
-        return Collections.unmodifiableList(futures);
+    public synchronized void replaceFuture(final ScheduledFuture<?> oldFuture, final ScheduledFuture<?> newFuture) {
+        futures.remove(oldFuture);
+        futures.add(newFuture);
+    }
+    
+    public synchronized Set<ScheduledFuture<?>> getFutures() {
+        return Collections.unmodifiableSet(futures);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4cc106a5/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
index db06151..efa8acd 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
@@ -18,8 +18,10 @@ package org.apache.nifi.controller.scheduling;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.ConnectableType;
@@ -31,15 +33,17 @@ import org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask;
 import org.apache.nifi.controller.tasks.ReportingTaskWrapper;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.engine.FlowEngine;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.StandardProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.FormatUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class TimerDrivenSchedulingAgent implements SchedulingAgent {
-
     private static final Logger logger = LoggerFactory.getLogger(TimerDrivenSchedulingAgent.class);
-
+    private static final long NO_WORK_YIELD_NANOS = TimeUnit.MILLISECONDS.toNanos(10L);
+    
     private final FlowController flowController;
     private final FlowEngine flowEngine;
     private final ProcessContextFactory contextFactory;
@@ -72,20 +76,95 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
         logger.info("{} started.", taskNode.getReportingTask());
     }
 
+    
     @Override
     public void schedule(final Connectable connectable, final ScheduleState scheduleState) {
-        final Runnable runnable;
-        if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
-            final ProcessorNode procNode = (ProcessorNode) connectable;
-            ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, encryptor);
-            runnable = runnableTask;
-        } else {
-            runnable = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, encryptor);
-        }
-
+        
         final List<ScheduledFuture<?>> futures = new ArrayList<>();
         for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) {
-            final ScheduledFuture<?> future = flowEngine.scheduleWithFixedDelay(runnable, 0L, connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+            final Callable<Boolean> continuallyRunTask;
+            final ProcessContext processContext;
+            
+            // Determine the task to run and create it.
+            if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
+                final ProcessorNode procNode = (ProcessorNode) connectable;
+                final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor);
+                final ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, 
+                        contextFactory, scheduleState, standardProcContext);
+                
+                continuallyRunTask = runnableTask;
+                processContext = standardProcContext;
+            } else {
+                processContext = new ConnectableProcessContext(connectable, encryptor);
+                continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, processContext);
+            }
+            
+            final AtomicReference<ScheduledFuture<?>> futureRef = new AtomicReference<>();
+            
+            final Runnable yieldDetectionRunnable = new Runnable() {
+                @Override
+                public void run() {
+                    // Call the continually run task. It will return a boolean indicating whether or not we should yield
+                    // based on a lack of work for to do for the component.
+                    final boolean shouldYield;
+                    try {
+                        shouldYield = continuallyRunTask.call();
+                    } catch (final RuntimeException re) {
+                        throw re;
+                    } catch (final Exception e) {
+                        throw new ProcessException(e);
+                    }
+                    
+                    // If the component is yielded, cancel its future and re-submit it to run again
+                    // after the yield has expired.
+                    final long newYieldExpiration = connectable.getYieldExpiration();
+                    if ( newYieldExpiration > System.currentTimeMillis() ) {
+                        final long yieldMillis = System.currentTimeMillis() - newYieldExpiration;
+                        final ScheduledFuture<?> scheduledFuture = futureRef.get();
+                        if ( scheduledFuture == null ) {
+                            return;
+                        }
+                        
+                        // If we are able to cancel the future, create a new one and update the ScheduleState so that it has
+                        // an accurate accounting of which futures are outstanding; we must then also update the futureRef
+                        // so that we can do this again the next time that the component is yielded.
+                        if (scheduledFuture.cancel(false)) {
+                            final long yieldNanos = TimeUnit.MILLISECONDS.toNanos(yieldMillis);
+                            final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, yieldNanos, 
+                                    connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+                            scheduleState.replaceFuture(scheduledFuture, newFuture);
+                            futureRef.set(newFuture);
+                        }
+                    } else if ( shouldYield ) {
+                        // Component itself didn't yield but there was no work to do, so the framework will choose
+                        // to yield the component automatically for a short period of time.
+                        final ScheduledFuture<?> scheduledFuture = futureRef.get();
+                        if ( scheduledFuture == null ) {
+                            return;
+                        }
+                        
+                        // If we are able to cancel the future, create a new one and update the ScheduleState so that it has
+                        // an accurate accounting of which futures are outstanding; we must then also update the futureRef
+                        // so that we can do this again the next time that the component is yielded.
+                        if (scheduledFuture.cancel(false)) {
+                            final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, NO_WORK_YIELD_NANOS, 
+                                    connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+                            scheduleState.replaceFuture(scheduledFuture, newFuture);
+                            futureRef.set(newFuture);
+                        }
+                    }
+                }
+            };
+
+            // Schedule the task to run
+            final ScheduledFuture<?> future = flowEngine.scheduleWithFixedDelay(yieldDetectionRunnable, 0L, 
+                    connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+            
+            // now that we have the future, set the atomic reference so that if the component is yielded we
+            // are able to then cancel this future.
+            futureRef.set(future);
+            
+            // Keep track of the futures so that we can update the ScheduleState.
             futures.add(future);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4cc106a5/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
index aca870b..408032c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
@@ -16,16 +16,16 @@
  */
 package org.apache.nifi.controller.tasks;
 
+import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.controller.repository.StandardProcessSessionFactory;
-import org.apache.nifi.controller.scheduling.ConnectableProcessContext;
 import org.apache.nifi.controller.scheduling.ProcessContextFactory;
 import org.apache.nifi.controller.scheduling.ScheduleState;
-import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.Connectables;
@@ -33,28 +33,33 @@ import org.apache.nifi.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class ContinuallyRunConnectableTask implements Runnable {
+/**
+ * Continually runs a Connectable as long as the processor has work to do. {@link #call()} will return
+ * <code>true</code> if the Connectable should be yielded, <code>false</code> otherwise.
+ */
+public class ContinuallyRunConnectableTask implements Callable<Boolean> {
 
     private static final Logger logger = LoggerFactory.getLogger(ContinuallyRunConnectableTask.class);
 
     private final Connectable connectable;
     private final ScheduleState scheduleState;
     private final ProcessSessionFactory sessionFactory;
-    private final ConnectableProcessContext processContext;
+    private final ProcessContext processContext;
 
-    public ContinuallyRunConnectableTask(final ProcessContextFactory contextFactory, final Connectable connectable, final ScheduleState scheduleState, final StringEncryptor encryptor) {
+    public ContinuallyRunConnectableTask(final ProcessContextFactory contextFactory, final Connectable connectable, final ScheduleState scheduleState, final ProcessContext processContext) {
         this.connectable = connectable;
         this.scheduleState = scheduleState;
         this.sessionFactory = new StandardProcessSessionFactory(contextFactory.newProcessContext(connectable, new AtomicLong(0L)));
-        this.processContext = new ConnectableProcessContext(connectable, encryptor);
+        this.processContext = processContext;
     }
 
-    @SuppressWarnings("deprecation")
     @Override
-    public void run() {
+    @SuppressWarnings("deprecation")
+    public Boolean call() {
         if (!scheduleState.isScheduled()) {
-            return;
+            return false;
         }
+        
         // Connectable should run if the following conditions are met:
         // 1. It's an Input Port or or is a Remote Input Port or has incoming FlowFiles queued
         // 2. Any relationship is available (since there's only 1
@@ -62,8 +67,9 @@ public class ContinuallyRunConnectableTask implements Runnable {
         // it means the same thing)
         // 3. It is not yielded.
         final boolean triggerWhenEmpty = connectable.isTriggerWhenEmpty();
+        boolean flowFilesQueued = true;
         final boolean shouldRun = (connectable.getYieldExpiration() < System.currentTimeMillis())
-                && (triggerWhenEmpty || Connectables.flowFilesQueued(connectable)) && (connectable.getRelationships().isEmpty() || Connectables.anyRelationshipAvailable(connectable));
+                && (triggerWhenEmpty || (flowFilesQueued = Connectables.flowFilesQueued(connectable))) && (connectable.getRelationships().isEmpty() || Connectables.anyRelationshipAvailable(connectable));
 
         if (shouldRun) {
             scheduleState.incrementActiveThreadCount();
@@ -92,6 +98,12 @@ public class ContinuallyRunConnectableTask implements Runnable {
 
                 scheduleState.decrementActiveThreadCount();
             }
+        } else if (!flowFilesQueued) {
+            // FlowFiles must be queued in order to run but there are none queued;
+            // yield for just a bit.
+            return true;
         }
+        
+        return true;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4cc106a5/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
index 33bd327..f4be855 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.controller.tasks;
 
 import java.io.IOException;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -31,7 +32,6 @@ import org.apache.nifi.controller.repository.StandardProcessSessionFactory;
 import org.apache.nifi.controller.scheduling.ProcessContextFactory;
 import org.apache.nifi.controller.scheduling.ScheduleState;
 import org.apache.nifi.controller.scheduling.SchedulingAgent;
-import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.ProcessSessionFactory;
@@ -43,7 +43,12 @@ import org.apache.nifi.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class ContinuallyRunProcessorTask implements Runnable {
+
+/**
+ * Continually runs a processor as long as the processor has work to do. {@link #call()} will return
+ * <code>true</code> if the processor should be yielded, <code>false</code> otherwise.
+ */
+public class ContinuallyRunProcessorTask implements Callable<Boolean> {
 
     private static final Logger logger = LoggerFactory.getLogger(ContinuallyRunProcessorTask.class);
 
@@ -56,7 +61,8 @@ public class ContinuallyRunProcessorTask implements Runnable {
     private final int numRelationships;
 
     public ContinuallyRunProcessorTask(final SchedulingAgent schedulingAgent, final ProcessorNode procNode,
-            final FlowController flowController, final ProcessContextFactory contextFactory, final ScheduleState scheduleState, final StringEncryptor encryptor) {
+            final FlowController flowController, final ProcessContextFactory contextFactory, final ScheduleState scheduleState, 
+            final StandardProcessContext processContext) {
 
         this.schedulingAgent = schedulingAgent;
         this.procNode = procNode;
@@ -65,28 +71,28 @@ public class ContinuallyRunProcessorTask implements Runnable {
         this.flowController = flowController;
 
         context = contextFactory.newProcessContext(procNode, new AtomicLong(0L));
-        this.processContext = new StandardProcessContext(procNode, flowController, encryptor);
+        this.processContext = processContext;
     }
 
-    @SuppressWarnings("deprecation")
     @Override
-    public void run() {
+    @SuppressWarnings("deprecation")
+    public Boolean call() {
         // make sure processor is not yielded
         boolean shouldRun = (procNode.getYieldExpiration() < System.currentTimeMillis());
         if (!shouldRun) {
-            return;
+            return false;
         }
 
         // make sure that either we're not clustered or this processor runs on all nodes or that this is the primary node
         shouldRun = !procNode.isIsolated() || !flowController.isClustered() || flowController.isPrimary();
         if (!shouldRun) {
-            return;
+            return false;
         }
 
         // make sure that either proc has incoming FlowFiles or has no incoming connections or is annotated with @TriggerWhenEmpty
         shouldRun = procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || Connectables.flowFilesQueued(procNode);
         if (!shouldRun) {
-            return;
+            return true;
         }
 
         if (numRelationships > 0) {
@@ -109,7 +115,7 @@ public class ContinuallyRunProcessorTask implements Runnable {
         }
 
         if (!shouldRun) {
-            return;
+            return false;
         }
 
         scheduleState.incrementActiveThreadCount();
@@ -124,11 +130,11 @@ public class ContinuallyRunProcessorTask implements Runnable {
                     invocationCount++;
 
                     if (!batch) {
-                        return;
+                        return false;
                     }
 
                     if (System.nanoTime() > finishNanos) {
-                        return;
+                        return false;
                     }
 
                     shouldRun = procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || Connectables.flowFilesQueued(procNode);
@@ -180,6 +186,8 @@ public class ContinuallyRunProcessorTask implements Runnable {
                 logger.error("", e);
             }
         }
+        
+        return false;
     }
 
 }


[28/29] incubator-nifi git commit: NIFI-335: Fixed bug that returned a byte instead of an int from CompressionInputStream.read(), which resulted in negative values getting returned when it should not have

Posted by ma...@apache.org.
NIFI-335: Fixed bug that returned a byte instead of an int from CompressionInputStream.read(), which resulted in negative values getting returned when it should not have


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a00aade0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a00aade0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a00aade0

Branch: refs/heads/develop
Commit: a00aade0e5b6f568104d207e38dbdc3d091416f5
Parents: 40e1fcb
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Feb 22 11:46:00 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Feb 22 11:46:00 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/remote/io/CompressionInputStream.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a00aade0/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
index 71cf894..d953185 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
@@ -142,7 +142,7 @@ public class CompressionInputStream extends InputStream {
             return -1;
         }
 
-        return buffer[bufferIndex++];
+        return buffer[bufferIndex++] & 0xFF;
     }
 
     @Override


[14/29] incubator-nifi git commit: NIFI-282: Fixed bug that caused client not to be able to communicate with remote NiFi instance

Posted by ma...@apache.org.
NIFI-282: Fixed bug that caused client not to be able to communicate with remote NiFi instance


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5c8a9e22
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5c8a9e22
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5c8a9e22

Branch: refs/heads/develop
Commit: 5c8a9e22d11007487b00d42455bc630451c76f82
Parents: d1e058c
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Feb 12 09:15:07 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Feb 12 09:15:07 2015 -0500

----------------------------------------------------------------------
 .../client/socket/EndpointConnectionPool.java   | 27 +++++++++----
 .../nifi/remote/client/socket/SocketClient.java | 12 ++++--
 .../client/socket/TestSiteToSiteClient.java     | 41 ++++++++++----------
 3 files changed, 47 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5c8a9e22/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index 43bc8e5..c0e4761 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -38,6 +38,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -114,6 +115,7 @@ public class EndpointConnectionPool {
     private final SSLContext sslContext;
     private final ScheduledExecutorService taskExecutor;
     private final int idleExpirationMillis;
+    private final RemoteDestination remoteDestination;
     
     private final ReadWriteLock listeningPortRWLock = new ReentrantReadWriteLock();
     private final Lock remoteInfoReadLock = listeningPortRWLock.readLock();
@@ -128,15 +130,17 @@ public class EndpointConnectionPool {
     private volatile boolean shutdown = false;
     
     
-    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final int idleExpirationMillis, 
-            final EventReporter eventReporter, final File persistenceFile) 
+    public EndpointConnectionPool(final String clusterUrl, final RemoteDestination remoteDestination, final int commsTimeoutMillis, 
+            final int idleExpirationMillis, final EventReporter eventReporter, final File persistenceFile) 
     {
-    	this(clusterUrl, commsTimeoutMillis, idleExpirationMillis, null, eventReporter, persistenceFile);
+    	this(clusterUrl, remoteDestination, commsTimeoutMillis, idleExpirationMillis, null, eventReporter, persistenceFile);
     }
     
-    public EndpointConnectionPool(final String clusterUrl, final int commsTimeoutMillis, final int idleExpirationMillis,
+    public EndpointConnectionPool(final String clusterUrl, final RemoteDestination remoteDestination, final int commsTimeoutMillis, final int idleExpirationMillis,
             final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile) 
     {
+        Objects.requireNonNull(clusterUrl, "URL cannot be null");
+        Objects.requireNonNull(remoteDestination, "Remote Destination/Port Identifier cannot be null");
     	try {
     		this.clusterUrl = new URI(clusterUrl);
     	} catch (final URISyntaxException e) {
@@ -150,6 +154,7 @@ public class EndpointConnectionPool {
         }
         apiUri = this.clusterUrl.getScheme() + "://" + this.clusterUrl.getHost() + ":" + this.clusterUrl.getPort() + uriPath + "-api";
         
+        this.remoteDestination = remoteDestination;
     	this.sslContext = sslContext;
     	this.peersFile = persistenceFile;
     	this.eventReporter = eventReporter;
@@ -197,12 +202,12 @@ public class EndpointConnectionPool {
     }
     
     
-    public EndpointConnection getEndpointConnection(final RemoteDestination remoteDestination, final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
-        return getEndpointConnection(remoteDestination, direction, null);
+    public EndpointConnection getEndpointConnection(final TransferDirection direction) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
+        return getEndpointConnection(direction, null);
     }
     
     
-    public EndpointConnection getEndpointConnection(final RemoteDestination remoteDestination, final TransferDirection direction, final SiteToSiteClientConfig config) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
+    public EndpointConnection getEndpointConnection(final TransferDirection direction, final SiteToSiteClientConfig config) throws IOException, HandshakeException, PortNotRunningException, UnknownPortException, ProtocolException {
     	//
         // Attempt to get a connection state that already exists for this URL.
         //
@@ -419,6 +424,7 @@ public class EndpointConnectionPool {
         return (peerList == null || peerList.isEmpty() || System.currentTimeMillis() > peerRefreshTime + PEER_REFRESH_PERIOD);
     }
     
+    
     private PeerStatus getNextPeerStatus(final TransferDirection direction) {
         List<PeerStatus> peerList = peerStatuses;
         if ( isPeerRefreshNeeded(peerList) ) {
@@ -532,7 +538,12 @@ public class EndpointConnectionPool {
         RemoteResourceInitiator.initiateResourceNegotiation(clientProtocol, dis, dos);
 
         clientProtocol.setTimeout(commsTimeout);
-        clientProtocol.handshake(peer, null);
+        if (clientProtocol.getVersionNegotiator().getVersion() < 5) {
+            clientProtocol.handshake(peer, remoteDestination.getIdentifier());
+        } else {
+            clientProtocol.handshake(peer, null);
+        }
+        
         final Set<PeerStatus> peerStatuses = clientProtocol.getPeerStatuses(peer);
         persistPeerStatuses(peerStatuses);
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5c8a9e22/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
index aae19b3..016e67f 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -40,9 +40,11 @@ public class SocketClient implements SiteToSiteClient {
 	private final String portName;
 	private final long penalizationNanos;
 	private volatile String portIdentifier;
+	private volatile boolean closed = false;
 	
 	public SocketClient(final SiteToSiteClientConfig config) {
-		pool = new EndpointConnectionPool(config.getUrl(), (int) config.getTimeout(TimeUnit.MILLISECONDS),
+		pool = new EndpointConnectionPool(config.getUrl(), createRemoteDestination(config.getPortIdentifier()), 
+		        (int) config.getTimeout(TimeUnit.MILLISECONDS),
 		        (int) config.getIdleConnectionExpiration(TimeUnit.MILLISECONDS),
 				config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile());
 		
@@ -107,15 +109,16 @@ public class SocketClient implements SiteToSiteClient {
 	
 	@Override
 	public Transaction createTransaction(final TransferDirection direction) throws IOException {
+	    if ( closed ) {
+	        throw new IllegalStateException("Client is closed");
+	    }
 		final String portId = getPortIdentifier(direction);
 		
 		if ( portId == null ) {
 			throw new IOException("Could not find Port with name '" + portName + "' for remote NiFi instance");
 		}
 		
-		final RemoteDestination remoteDestination = createRemoteDestination(portId);
-		
-		final EndpointConnection connectionState = pool.getEndpointConnection(remoteDestination, direction, getConfig());
+		final EndpointConnection connectionState = pool.getEndpointConnection(direction, getConfig());
 		if ( connectionState == null ) {
 		    return null;
 		}
@@ -196,6 +199,7 @@ public class SocketClient implements SiteToSiteClient {
 	
 	@Override
 	public void close() throws IOException {
+	    closed = true;
 		pool.shutdown();
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5c8a9e22/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
index 2fd90f8..75becd3 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestSiteToSiteClient.java
@@ -75,28 +75,27 @@ public class TestSiteToSiteClient {
     public void testSend() throws IOException {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.remote", "DEBUG");
         
-        final SiteToSiteClient client = new SiteToSiteClient.Builder()
-            .url("http://10.0.64.63:8080/nifi")
-            .portName("input")
-            .nodePenalizationPeriod(10, TimeUnit.MILLISECONDS)
-            .build();
+            final SiteToSiteClient client = new SiteToSiteClient.Builder()
+                .url("http://localhost:8080/nifi")
+                .portName("input")
+                .build();
         
-        try {
-            final Transaction transaction = client.createTransaction(TransferDirection.SEND);
-            Assert.assertNotNull(transaction);
-            
-            final Map<String, String> attrs = new HashMap<>();
-            attrs.put("site-to-site", "yes, please!");
-            final byte[] bytes = "Hello".getBytes();
-            final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-            final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
-            transaction.send(packet);
-
-            transaction.confirm();
-            transaction.complete();
-        } finally {
-            client.close();
-        }
+            try {
+                final Transaction transaction = client.createTransaction(TransferDirection.SEND);
+                Assert.assertNotNull(transaction);
+                
+                    final Map<String, String> attrs = new HashMap<>();
+                    attrs.put("site-to-site", "yes, please!");
+                    final byte[] bytes = "Hello".getBytes();
+                    final ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+                    final DataPacket packet = new StandardDataPacket(attrs, bais, bytes.length);
+                    transaction.send(packet);
+                
+                transaction.confirm();
+                transaction.complete();
+            } finally {
+                client.close();
+            }
     }
     
 }