You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2015/02/26 04:54:04 UTC

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

Repository: incubator-nifi
Updated Branches:
  refs/heads/NIFI-353 cc0b6fe2f -> 48192289e


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>


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

Posted by mc...@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/NIFI-353
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();
                 }
             }


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

Posted by mc...@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/NIFI-353
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();
+            }
     }
     
 }


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

Posted by mc...@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


[42/51] [abbrv] incubator-nifi git commit: NIFI-370 refines system requirements in Admin Guide and fixes Clustering section in Admin Guide and Clustering description in the Overview document.

Posted by mc...@apache.org.
NIFI-370 refines system requirements in Admin Guide and fixes Clustering section in Admin Guide and Clustering description in the Overview document.


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

Branch: refs/heads/NIFI-353
Commit: c5f4dff4bbf2de1c23b5743083845800f0eaccec
Parents: 01038f4
Author: Jenn Barnabee <je...@gmail.com>
Authored: Tue Feb 24 08:17:39 2015 -0500
Committer: Jenn Barnabee <je...@gmail.com>
Committed: Tue Feb 24 08:17:39 2015 -0500

----------------------------------------------------------------------
 .../src/main/asciidoc/administration-guide.adoc | 27 ++++++++++----------
 nifi/nifi-docs/src/main/asciidoc/overview.adoc  |  4 +--
 2 files changed, 16 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c5f4dff4/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 938b581..4c4df30 100644
--- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -21,9 +21,9 @@ Apache NiFi Team <de...@nifi.incubator.apache.org>
 
 System Requirements
 -------------------
-Apache NiFi can run on something as simple as a laptop, but it can also be clustered across many enterprise servers. The hardware and memory you need will depend on the size and nature of the dataflow you are running. NiFi has the following system requirements:
+Apache NiFi can run on something as simple as a laptop, but it can also be clustered across many enterprise-class servers. Therefore, the amount of hardware and memory needed will depend on the size and nature of the dataflow involved. The data is stored on disk while NiFi is processing it. So NiFi needs to have sufficient disk space allocated for its various repositories, particularly the content repository, flowfile repository, and provenance repository (see the <<system_properties>> section for more information about these repositories). NiFi has the following minimum system requirements:
 
-* Requires Java 7
+* Requires Java 7 or newer
 * Supported Operating Systems: 
 ** Linux
 ** Unix
@@ -40,7 +40,7 @@ Note that there is a known issue in Internet Explorer (IE) 10 and 11 that can ca
 How to install and start NiFi
 -----------------------------
 
-* Linux/Unix/OSX
+* Linux/Unix/OS X
 ** Decompress and untar into desired installation directory
 ** Make any desired edits in files found under <installdir>/conf
 *** At a minimum, we recommend editing the _nifi.properties_ file and entering a password for the nifi.sensitive.props.key (see <<system_properties>> below)
@@ -78,10 +78,9 @@ When NiFi first starts up, the following files and directories are created:
 See the <<system_properties>> section of this guide for more information about configuring NiFi repositories and configuration files.
 
 
-Best Practice Configuration
----------------------------
-NOTE: Typical Linux defaults are not necessarily well tuned for the needs of an IO intensive application like
-NiFi.  For all of these areas your distribution's requirements may vary.  Use these sections as advice but
+Configuration Best Practices
+----------------------------
+NOTE: If you are running on Linux, consider these best practices. Typical Linux defaults are not necessarily well tuned for the needs of an IO intensive application like NiFi.  For all of these areas, your distribution's requirements may vary.  Use these sections as advice, but
 consult your distribution-specific documentation for how best to achieve these recommendations.
 
 Maximum File Handles::
@@ -201,7 +200,7 @@ Clustering Configuration
 
 This section provides a quick overview of NiFi Clustering and instructions on how to set up a basic cluster. In the future, we hope to provide supplemental documentation that covers the NiFi Cluster Architecture in depth. 
 
-The design of NiFi clustering is a simple master/slave model where there is a master and one or more slaves. While the model is that of master and slave, if the master dies, the slaves are all instructed to continue operating as they were to ensure the dataflow remains live. The absence of the master simply means new slaves cannot come on-line and flow changes cannot occur until the master is restored. In NiFi clustering, we call the master the NiFi Cluster Manager (NCM), and the slaves are called Nodes. See a full description of each in the Terminology section below.
+The design of NiFi clustering is a simple master/slave model where there is a master and one or more slaves. While the model is that of master and slave, if the master dies, the slaves are all instructed to continue operating as they were to ensure the dataflow remains live. The absence of the master simply means new slaves cannot join the cluster and cluster flow changes cannot occur until the master is restored. In NiFi clustering, we call the master the NiFi Cluster Manager (NCM), and the slaves are called Nodes. See a full description of each in the Terminology section below.
 
 *Why Cluster?* +
 
@@ -216,17 +215,17 @@ NiFi Clustering is unique and has its own terminology. It's important to underst
 
 *Nodes*: Each cluster is made up of the NCM and one or more nodes. The nodes do the actual data processing. (The NCM does not process any data; all data runs through the nodes.)  While nodes are connected to a cluster, the DFM may not access the User Interface for any of the individual nodes. The User Interface of a node may only be accessed if the node is manually removed from the cluster.
 
-*Primary Node*: Every cluster has one Primary Node. On this node, it is possible to run "Isolated Processors" (see below). By default, the NCM will elect the first node that connects to the cluster as the Primary Node; however, the DFM may select a new node as the Primary Node in the Cluster Management page of the User Interface if desired. If the cluster restarts, the NCM will "remember" which node was he Primary Node and wait for that node to re-connect before allowing the DFM to make any changes to the dataflow. The ADMIN may adjust how long the NCM waits for the Primary Node to reconnect by adjusting the property _nifi.cluster.manager.safemode.duration_ in the _nifi.properties_ file, which is discussed in the <<system_properties>> section of this document. 
+*Primary Node*: Every cluster has one Primary Node. On this node, it is possible to run "Isolated Processors" (see below). By default, the NCM will elect the first node that connects to the cluster as the Primary Node; however, the DFM may select a new node as the Primary Node in the Cluster Management page of the User Interface if desired. If the cluster restarts, the NCM will "remember" which node was the Primary Node and wait for that node to re-connect before allowing the DFM to make any changes to the dataflow. The ADMIN may adjust how long the NCM waits for the Primary Node to reconnect by adjusting the property _nifi.cluster.manager.safemode.duration_ in the _nifi.properties_ file, which is discussed in the <<system_properties>> section of this document. 
 
-*Isolated Processors*: In a NiFi cluster, the same dataflow runs on all the nodes. As a result, every component in the flow runs on every node. However, there may be cases when the DFM would not want every processor to run on every node. The most common case is when using a processor like the GetSFTP processor, which is pulling from a remote directory. If the GetSFTP on every node tries simultaneously to pull from the same remote directory, there could be race conditions. Therefore, the DFM could configure the GetSFTP on the Primary Node to run in isolation, meaning that it only runs on that node. It could pull in data and -with the proper dataflow configuration- load-balance it across the rest of the nodes in the cluster. Note that while this feature exists, it is also very common to simply use a standalone NiFi instance to pull data and feed it to the cluster. It just depends on the resources available and how the Administrator decides to configure the cluster. 
+*Isolated Processors*: In a NiFi cluster, the same dataflow runs on all the nodes. As a result, every component in the flow runs on every node. However, there may be cases when the DFM would not want every processor to run on every node. The most common case is when using a processor that communicates with an external service using a protocol that does not scale well. For example, the GetSFTP processor pulls from a remote directory, and if the GetSFTP on every node in the cluster tries simultaneously to pull from the same remote directory, there could be race conditions. Therefore, the DFM could configure the GetSFTP on the Primary Node to run in isolation, meaning that it only runs on that node. It could pull in data and -with the proper dataflow configuration- load-balance it across the rest of the nodes in the cluster. Note that while this feature exists, it is also very common to simply use a standalone NiFi instance to pull data and feed it to the cluster. It just depends on th
 e resources available and how the Administrator decides to configure the cluster. 
 
 *Heartbeats*: The nodes communicate their health and status to the NCM via "heartbeats", which let the NCM know they are still connected to the cluster and working properly. By default, the nodes emit heartbeats to the NCM every 5 seconds, and if the NCM does not receive a heartbeat from a node within 45 seconds, it disconnects the node due to "lack of heartbeat". (The 5-second and 45-second settings are configurable in the _nifi.properties_ file. See the <<system_properties>> section of this document for more information.) The reason that the NCM disconnects the node is because the NCM needs to ensure that every node in the cluster is in sync, and if a node is not heard from regularly, the NCM cannot be sure it is still in sync with the rest of the cluster. If, after 45 seconds, the node does send a new heartbeat, the NCM will automatically reconnect the node to the cluster. Both the disconnection due to lack of heartbeat and the reconnection once a heartbeat is received are report
 ed to the DFM in the NCM's User Interface. 
 
 *Communication within the Cluster* +
 
-As noted, the nodes communicate with the NCM via heartbeats. The NCM-to-node communication may be set up as multicast or unicast, depending on the properties that are configured in the _nifi.properties_ file (See <<system_properties>> ). By default, unicast is used. It is important to note that the nodes in a NiFi cluster are not aware of each other. They only communicate with the NCM. Therefore, if one of the nodes goes down, the other nodes in the cluster will not automatically pick up the load of the missing node. It is possible for the DFM to configure the dataflow for failover contingencies; however, this is dependent on the dataflow design and does not happen automatically.
+As noted, the nodes communicate with the NCM via heartbeats. The communication that allows the nodes to find the NCM may be set up as multicast or unicast; this is configured in the _nifi.properties_ file (See <<system_properties>> ). By default, unicast is used. It is important to note that the nodes in a NiFi cluster are not aware of each other. They only communicate with the NCM. Therefore, if one of the nodes goes down, the other nodes in the cluster will not automatically pick up the load of the missing node. It is possible for the DFM to configure the dataflow for failover contingencies; however, this is dependent on the dataflow design and does not happen automatically.
 
-When the DFM makes changes to the dataflow, the NCM communicates those changes to the nodes and waits for each node to respond, indicating that it has made the change on its local flow. If the DFM wants to make another change, the NCM will only allow this to happen once all the nodes have acknowledged that they've implemented the last change. As such, the speed with which dataflow changes may be made is as fast as the slowest node. When all nodes are located in close proximity and the network is stable, this response time is not an issue. However, if your cluster is comprised of nodes that are geographically dispersed and/or operating over a latent network, there may be times when DFMs cannot make changes as quickly as they would like. Keep this in mind when setting up a cluster.
+When the DFM makes changes to the dataflow, the NCM communicates those changes to the nodes and waits for each node to respond, indicating that it has made the change on its local flow. If the DFM wants to make another change, the NCM will only allow this to happen once all the nodes have acknowledged that they've implemented the last change. This is a safeguard to ensure that all the nodes in the cluster have the correct and up-to-date flow.
 
 *Dealing with Disconnected Nodes* +
 
@@ -259,7 +258,8 @@ For Node 1, the minimum properties to configure are as follows:
 * Under the Web Properties, set either the http or https port that you want Node 1 to run on. If the NCM is running on the same server, choose a different web port for Node 1.
 * Under Cluster Node Properties, set the following:
 ** nifi.cluster.is.node - Set this to _true_.
-** nifi.cluster.node.protocol.port - Set this to an open port that is higher than 1024 (anything lower requires root). If Node 1 and the NCM are on the same server, make sure this port is different from the nifi.cluster.protocol.manager.port.  
+** nifi.cluster.node.protocol.port - Set this to an open port that is higher than 1024 (anything lower requires root). If Node 1 and the NCM are on the same server, make sure this port is different from the nifi.cluster.protocol.manager.port.
+** nifi.cluster.node.unicast.manager.address - Set this to the NCM's fully qualified hostname.  
 ** nifi.cluster.node.unicast.manager.protocol.port - Set this to exactly the same port that was set on the NCM for the property nifi.cluster.manager.protocol.port.
 
 For Node 2, the minimum properties to configure are as follows:
@@ -268,6 +268,7 @@ For Node 2, the minimum properties to configure are as follows:
 * Under the Cluster Node Properties, set the following:
 ** nifi.cluster.is.node - Set this to _true_.
 ** nifi.cluster.node.protocol.port - Set this to an open port that is higher than 1024 (anything lower requires root).
+** nifi.cluster.node.unicast.manager.address - Set this to the NCM's fully qualified hostname.
 ** nifi.cluster.node.unicast.manager.protocol.port - Set this to exactly the same port that was set on the NCM for the property nifi.cluster.manager.protocol.port.
 
 Now, it is possible to start up the cluster. Technically, it does not matter which instance starts up first. However, you could start the NCM first, then Node 1 and then Node 2. Since the first node that connects is automatically elected as the Primary Node, this sequence should create a cluster where Node 1 is the Primary Node. Navigate to the URL for the NCM in your web browser, and the User Interface should look similar to the following:

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c5f4dff4/nifi/nifi-docs/src/main/asciidoc/overview.adoc
----------------------------------------------------------------------
diff --git a/nifi/nifi-docs/src/main/asciidoc/overview.adoc b/nifi/nifi-docs/src/main/asciidoc/overview.adoc
index a5183d7..2e62649 100644
--- a/nifi/nifi-docs/src/main/asciidoc/overview.adoc
+++ b/nifi/nifi-docs/src/main/asciidoc/overview.adoc
@@ -155,10 +155,10 @@ by a single NiFi Cluster Manager (NCM).  The design of clustering is a simple
 master/slave model where the NCM is the master and the Nodes are the slaves.
 The NCM's reason for existence is to keep track of which Nodes are in the cluster,
 their status, and to replicate requests to modify or observe the 
-flow.  Fundamentally then the NCM keeps the state of the cluster consistent.  
+flow.  Fundamentally, then, the NCM keeps the state of the cluster consistent.  
 While the model is that of master and slave, if the master dies the Nodes are all
 instructed to continue operating as they were to ensure the data flow remains live.
-The absence of the NCM simply means new nodes cannot come on-line and flow changes
+The absence of the NCM simply means new nodes cannot join the cluster and cluster flow changes
 cannot occur until the NCM is restored.
 
 Performance Expectations and Characteristics of NiFi


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

Posted by mc...@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>


[39/51] [abbrv] incubator-nifi git commit: NIFI-370 This adds system requirements to the Admin Guide and a list of supported browsers to the User Guide.

Posted by mc...@apache.org.
NIFI-370 This adds system requirements to the Admin Guide and a list of supported browsers to the User Guide.


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

Branch: refs/heads/NIFI-353
Commit: 01038f4e2c26c21c7443cffb8a7426596687a43e
Parents: b6f4026
Author: Jenn Barnabee <je...@gmail.com>
Authored: Mon Feb 23 14:08:47 2015 -0500
Committer: Jenn Barnabee <je...@gmail.com>
Committed: Mon Feb 23 14:08:47 2015 -0500

----------------------------------------------------------------------
 .../src/main/asciidoc/administration-guide.adoc | 22 ++++++++++++++++++--
 .../nifi-docs/src/main/asciidoc/user-guide.adoc | 11 ++++++++++
 2 files changed, 31 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/01038f4e/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
index b2fdc5a..938b581 100644
--- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -19,6 +19,24 @@ NiFi System Administrator's Guide
 Apache NiFi Team <de...@nifi.incubator.apache.org>
 :homepage: http://nifi.incubator.apache.org
 
+System Requirements
+-------------------
+Apache NiFi can run on something as simple as a laptop, but it can also be clustered across many enterprise servers. The hardware and memory you need will depend on the size and nature of the dataflow you are running. NiFi has the following system requirements:
+
+* Requires Java 7
+* Supported Operating Systems: 
+** Linux
+** Unix
+** Windows
+** Mac OS X
+* Supported Web Browsers: 
+** Internet Explorer 9+ (see note below)
+** Mozilla FireFox 24+
+** Google Chrome 36+
+** Safari 8
+
+Note that there is a known issue in Internet Explorer (IE) 10 and 11 that can cause problems when moving items on the NiFi graph. If you encounter this problem, we suggest using a browser other than IE. This known issue is described here: https://connect.microsoft.com/IE/Feedback/Details/1050422.
+
 How to install and start NiFi
 -----------------------------
 
@@ -63,8 +81,8 @@ See the <<system_properties>> section of this guide for more information about c
 Best Practice Configuration
 ---------------------------
 NOTE: Typical Linux defaults are not necessarily well tuned for the needs of an IO intensive application like
-NiFi.  For all of these areas your distributions requirements may vary.  Use these sections as advice but
-consult your distribution specific documentation for how best to achieve these recommendations.
+NiFi.  For all of these areas your distribution's requirements may vary.  Use these sections as advice but
+consult your distribution-specific documentation for how best to achieve these recommendations.
 
 Maximum File Handles::
 NiFi will at any one time potentially have a very large number of file handles open.  Increase the limits by

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/01038f4e/nifi/nifi-docs/src/main/asciidoc/user-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/user-guide.adoc
index 9208153..e13aaac 100644
--- a/nifi/nifi-docs/src/main/asciidoc/user-guide.adoc
+++ b/nifi/nifi-docs/src/main/asciidoc/user-guide.adoc
@@ -29,6 +29,17 @@ along several dimensions of quality of service, such as loss-tolerant versus gua
 high throughput, and priority-based queuing. NiFi provides fine-grained data provenance for all data received, forked, joined
 cloned, modified, sent, and ultimately dropped upon reaching its configured end-state.
 
+See the Admin Guide for information about system requirements, installation, and configuration. Once NiFi is installed,
+use a supported web browser to view the User Interface. Supported web browsers include:
+
+* Internet Explorer 9+ (see note below)
+* Mozilla FireFox 24+
+* Google Chrome 36+
+* Safari 8
+
+Note that there is a known issue in Internet Explorer (IE) 10 and 11 that can cause problems when moving items on the NiFi graph. If you encounter this problem, we suggest using a browser other than IE. This known issue is described here: https://connect.microsoft.com/IE/Feedback/Details/1050422.
+
+
 
 [template="glossary", id="terminology"]
 Terminology


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

Posted by mc...@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);
         }


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

Posted by mc...@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);
     }


[19/51] [abbrv] incubator-nifi git commit: NIFI-333 - Adding failure test for PutEmail

Posted by mc...@apache.org.
NIFI-333 - Adding failure test for PutEmail

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

Branch: refs/heads/NIFI-353
Commit: 0fa1b16c831c413a2e31e768398da0602ca3758d
Parents: 361ac1f
Author: danbress <db...@onyxconsults.com>
Authored: Sun Feb 15 09:24:51 2015 -0500
Committer: danbress <db...@onyxconsults.com>
Committed: Sun Feb 15 09:24:51 2015 -0500

----------------------------------------------------------------------
 .../nifi/processors/standard/TestPutEmail.java  | 45 ++++++++++++++++++++
 1 file changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0fa1b16c/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java
new file mode 100644
index 0000000..b737ed6
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.standard;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Test;
+
+public class TestPutEmail {
+
+    @Test
+    public void testHotNotFound() {
+        // verifies that files are routed to failure when the SMTP host doesn't exist
+        final TestRunner runner = TestRunners.newTestRunner(new PutEmail());
+        runner.setProperty(PutEmail.SMTP_HOSTNAME, "host-doesnt-exist123");
+        runner.setProperty(PutEmail.FROM, "test@apache.org");
+        runner.setProperty(PutEmail.TO, "test@apache.org");
+        runner.setProperty(PutEmail.MESSAGE, "Message Body");
+
+        final Map<String, String> attributes = new HashMap<>();
+        runner.enqueue("Some Text".getBytes(), attributes);
+
+        runner.run();
+
+        runner.assertQueueEmpty();
+        runner.assertAllFlowFilesTransferred(PutEmail.REL_FAILURE);
+    }
+}


[32/51] [abbrv] 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 mc...@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/NIFI-353
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;
     }
 
 }


[20/51] [abbrv] 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 mc...@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/NIFI-353
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;


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

Posted by mc...@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/NIFI-353
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 {


[45/51] [abbrv] incubator-nifi git commit: NIFI-381: do not re-schedule processor to run after yield if not scheduled to run anymore

Posted by mc...@apache.org.
NIFI-381: do not re-schedule processor to run after yield if not scheduled to run anymore


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

Branch: refs/heads/NIFI-353
Commit: a956623ff92fe35aecabd95d6f6f2b85de6b3edc
Parents: e370d7d
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 25 11:38:28 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 25 11:38:28 2015 -0500

----------------------------------------------------------------------
 .../controller/scheduling/StandardProcessScheduler.java  |  5 ++---
 .../scheduling/TimerDrivenSchedulingAgent.java           | 11 ++++++-----
 2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a956623f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 1627994..4407451 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -43,7 +43,6 @@ import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.annotation.OnConfigured;
-import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.engine.FlowEngine;
@@ -374,9 +373,9 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                 return;
             }
 
+            state.setScheduled(false);
             getSchedulingAgent(procNode).unschedule(procNode, state);
             procNode.setScheduledState(ScheduledState.STOPPED);
-            state.setScheduled(false);
         }
 
         final Runnable stopProcRunnable = new Runnable() {
@@ -474,8 +473,8 @@ public final class StandardProcessScheduler implements ProcessScheduler {
         if (!state.isScheduled()) {
             return;
         }
+        
         state.setScheduled(false);
-
         getSchedulingAgent(connectable).unschedule(connectable, state);
 
         if (!state.isScheduled() && state.getActiveThreadCount() == 0 && state.mustCallOnStoppedMethods()) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a956623f/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 a620202..17fb9f8 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
@@ -130,11 +130,12 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
                         // 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);
                             
                             synchronized (scheduleState) {
                                 if ( scheduleState.isScheduled() ) {
+                                    final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, yieldNanos, 
+                                            connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+                                    
                                     scheduleState.replaceFuture(scheduledFuture, newFuture);
                                     futureRef.set(newFuture);
                                 }
@@ -152,11 +153,11 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
                         // 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);
-                            
                             synchronized (scheduleState) {
                                 if ( scheduleState.isScheduled() ) {
+                                    final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, NO_WORK_YIELD_NANOS, 
+                                            connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
+                                    
                                     scheduleState.replaceFuture(scheduledFuture, newFuture);
                                     futureRef.set(newFuture);
                                 }


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

Posted by mc...@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/NIFI-353
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
----------------------------------------------------------------------


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

Posted by mc...@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/NIFI-353
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;
-    }
-}


[51/51] [abbrv] incubator-nifi git commit: Merge branch 'develop' into NIFI-353

Posted by mc...@apache.org.
Merge branch 'develop' into NIFI-353


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

Branch: refs/heads/NIFI-353
Commit: 48192289e492ed98bcc433925a70203570223c2a
Parents: cc0b6fe 50744bf
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Feb 25 22:53:29 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Wed Feb 25 22:53:29 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/remote/RemoteDestination.java   |  53 +
 nifi/nifi-assembly/pom.xml                      |   1 +
 .../org/apache/nifi/util/NiFiProperties.java    |   5 +
 .../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   | 965 +++++++++++++++++++
 .../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     | 110 +++
 .../remote/io/socket/SocketChannelInput.java    |  71 ++
 .../remote/io/socket/SocketChannelOutput.java   |  58 ++
 .../SSLSocketChannelCommunicationsSession.java  | 113 +++
 .../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 ++
 .../nifi/remote/io/CompressionInputStream.java  |   2 +-
 .../io/socket/SocketChannelInputStream.java     |  12 +
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  12 +
 .../socket/ssl/SSLSocketChannelInputStream.java |   4 +
 .../nifi/stream/io/LimitingInputStream.java     | 111 +++
 .../stream/io/MinimumLengthInputStream.java     |  93 ++
 nifi/nifi-commons/pom.xml                       |   1 +
 .../src/main/asciidoc/administration-guide.adoc |  43 +-
 nifi/nifi-docs/src/main/asciidoc/overview.adoc  |   4 +-
 .../nifi-docs/src/main/asciidoc/user-guide.adoc |  11 +
 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/controller/StandardFunnel.java  |   2 +-
 .../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 +-
 .../scheduling/QuartzSchedulingAgent.java       |  21 +-
 .../controller/scheduling/ScheduleState.java    |  18 +-
 .../scheduling/StandardProcessScheduler.java    |   5 +-
 .../scheduling/TimerDrivenSchedulingAgent.java  | 124 ++-
 .../tasks/ContinuallyRunConnectableTask.java    |  32 +-
 .../tasks/ContinuallyRunProcessorTask.java      |  82 +-
 .../org/apache/nifi/remote/RemoteNiFiUtils.java | 216 +++++
 .../nifi/remote/StandardRemoteProcessGroup.java | 353 +------
 .../src/main/resources/conf/nifi.properties     |   2 +
 .../nifi-framework/nifi-site-to-site/.gitignore |   1 +
 .../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    | 697 ++++----------
 .../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 --
 .../web/api/RemoteProcessGroupResource.java     |   8 +
 .../canvas/new-remote-process-group-dialog.jsp  |   2 +-
 .../main/webapp/js/nf/canvas/nf-context-menu.js |   2 +-
 .../hadoop/CreateHadoopSequenceFile.java        |   4 +-
 .../standard/Base64EncodeContent.java           |  13 +-
 .../processors/standard/CompressContent.java    |  27 +-
 .../nifi/processors/standard/HashContent.java   |  14 +-
 .../nifi/processors/standard/PutEmail.java      |   5 +-
 .../processors/standard/SegmentContent.java     | 114 ++-
 .../nifi/processors/standard/TransformXml.java  |   3 +-
 .../standard/TestCompressContent.java           |  18 +
 .../nifi/processors/standard/TestPutEmail.java  |  45 +
 nifi/pom.xml                                    |   6 +
 139 files changed, 7307 insertions(+), 3623 deletions(-)
----------------------------------------------------------------------



[27/51] [abbrv] incubator-nifi git commit: NIFI-365: Initial implementation of spark receiver

Posted by mc...@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/NIFI-353
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>


[41/51] [abbrv] incubator-nifi git commit: Merge branch 'NIFI-333' of https://github.com/danbress/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'NIFI-333' of https://github.com/danbress/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/d8954ab0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d8954ab0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d8954ab0

Branch: refs/heads/NIFI-353
Commit: d8954ab0137b0375dce2be8b65adbe721fc74f1b
Parents: 705ee85 c8810c0
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Feb 23 15:34:13 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Feb 23 15:34:13 2015 -0500

----------------------------------------------------------------------
 .../hadoop/CreateHadoopSequenceFile.java        |   4 +-
 .../standard/Base64EncodeContent.java           |  13 ++-
 .../processors/standard/CompressContent.java    |  27 ++---
 .../nifi/processors/standard/HashContent.java   |  14 +--
 .../nifi/processors/standard/PutEmail.java      |   5 +-
 .../processors/standard/SegmentContent.java     | 114 +++++++++----------
 .../nifi/processors/standard/TransformXml.java  |   3 +-
 .../standard/TestCompressContent.java           |  18 +++
 .../nifi/processors/standard/TestPutEmail.java  |  45 ++++++++
 9 files changed, 152 insertions(+), 91 deletions(-)
----------------------------------------------------------------------



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

Posted by mc...@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;
+    }
+}


[48/51] [abbrv] incubator-nifi git commit: NIFI-381: Ensure that we always properly account for number of active threads

Posted by mc...@apache.org.
NIFI-381: Ensure that we always properly account for number of active threads


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

Branch: refs/heads/NIFI-353
Commit: 1af8c1e22a32b2e4024a655a31735be1d170b5df
Parents: ca23ad8
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 25 14:07:21 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 25 14:07:21 2015 -0500

----------------------------------------------------------------------
 .../tasks/ContinuallyRunProcessorTask.java      | 50 ++++++++++----------
 1 file changed, 26 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1af8c1e2/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 f4be855..cff8744 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
@@ -159,31 +159,33 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
                 procNode.yield(schedulingAgent.getAdministrativeYieldDuration(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
             }
         } finally {
-            if (batch) {
-                rawSession.commit();
-            }
-
-            final long processingNanos = System.nanoTime() - startNanos;
-
-            // if the processor is no longer scheduled to run and this is the last thread,
-            // invoke the OnStopped methods
-            if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) {
-                try (final NarCloseable x = NarCloseable.withNarLoader()) {
-                    ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, procNode.getProcessor(), processContext);
-                    flowController.heartbeat();
-                }
-            }
-
-            scheduleState.decrementActiveThreadCount();
-
             try {
-                final StandardFlowFileEvent procEvent = new StandardFlowFileEvent(procNode.getIdentifier());
-                procEvent.setProcessingNanos(processingNanos);
-                procEvent.setInvocations(invocationCount);
-                context.getFlowFileEventRepository().updateRepository(procEvent);
-            } catch (final IOException e) {
-                logger.error("Unable to update FlowFileEvent Repository for {}; statistics may be inaccurate. Reason for failure: {}", procNode.getProcessor(), e.toString());
-                logger.error("", e);
+                if (batch) {
+                    rawSession.commit();
+                }
+    
+                final long processingNanos = System.nanoTime() - startNanos;
+    
+                // if the processor is no longer scheduled to run and this is the last thread,
+                // invoke the OnStopped methods
+                if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) {
+                    try (final NarCloseable x = NarCloseable.withNarLoader()) {
+                        ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, procNode.getProcessor(), processContext);
+                        flowController.heartbeat();
+                    }
+                }
+    
+                try {
+                    final StandardFlowFileEvent procEvent = new StandardFlowFileEvent(procNode.getIdentifier());
+                    procEvent.setProcessingNanos(processingNanos);
+                    procEvent.setInvocations(invocationCount);
+                    context.getFlowFileEventRepository().updateRepository(procEvent);
+                } catch (final IOException e) {
+                    logger.error("Unable to update FlowFileEvent Repository for {}; statistics may be inaccurate. Reason for failure: {}", procNode.getProcessor(), e.toString());
+                    logger.error("", e);
+                }
+            } finally {
+                scheduleState.decrementActiveThreadCount();
             }
         }
         


[36/51] [abbrv] incubator-nifi git commit: Merge branch 'nifi-site-to-site-client' into develop

Posted by mc...@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/NIFI-353
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
----------------------------------------------------------------------


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

Posted by mc...@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/1c8d0057
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1c8d0057
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1c8d0057

Branch: refs/heads/NIFI-353
Commit: 1c8d0057a78ff848dab81f344d7a10cabd3a39b2
Parents: c0110aa bf97094
Author: Jenn Barnabee <je...@gmail.com>
Authored: Fri Feb 13 08:07:20 2015 -0500
Committer: Jenn Barnabee <je...@gmail.com>
Committed: Fri Feb 13 08:07:20 2015 -0500

----------------------------------------------------------------------
 .gitignore                                      |    7 +
 README.md                                       |   20 +-
 nifi-nar-maven-plugin/README.md                 |   14 +
 nifi-nar-maven-plugin/pom.xml                   |   10 +-
 nifi-site/Gruntfile.js                          |  138 ++
 nifi-site/LICENSE                               |  202 ++
 nifi-site/README.md                             |  119 +
 nifi-site/bower.json                            |   19 +
 nifi-site/config.rb                             |   24 +
 nifi-site/package.json                          |   24 +
 nifi-site/robots.txt                            |    4 +
 nifi-site/src/images/bgNifiLogo.png             |  Bin 0 -> 4232 bytes
 nifi-site/src/images/egg-logo.png               |  Bin 0 -> 8626 bytes
 nifi-site/src/images/feather-small.gif          |  Bin 0 -> 7129 bytes
 nifi-site/src/images/flow.png                   |  Bin 0 -> 503302 bytes
 .../src/images/niFi-logo-horizontal-scaled.png  |  Bin 0 -> 4462 bytes
 nifi-site/src/images/niFi-logo-horizontal.png   |  Bin 0 -> 13161 bytes
 nifi-site/src/images/nifi16.ico                 |  Bin 0 -> 1150 bytes
 nifi-site/src/images/nifiDrop.svg               |  217 ++
 nifi-site/src/includes/disclaimer.hbs           |   27 +
 nifi-site/src/includes/footer.hbs               |    6 +
 nifi-site/src/includes/header.hbs               |   22 +
 nifi-site/src/includes/topbar.hbs               |   71 +
 nifi-site/src/js/app.js                         |   17 +
 nifi-site/src/layouts/html.hbs                  |    3 +
 nifi-site/src/layouts/markdown.hbs              |   12 +
 nifi-site/src/pages/html/developer-guide.hbs    |    7 +
 nifi-site/src/pages/html/download.hbs           |   46 +
 nifi-site/src/pages/html/faq.hbs                |   33 +
 nifi-site/src/pages/html/index.hbs              |  210 ++
 nifi-site/src/pages/html/mailing_lists.hbs      |   70 +
 nifi-site/src/pages/html/overview.hbs           |    7 +
 nifi-site/src/pages/html/people.hbs             |  136 ++
 nifi-site/src/pages/html/roadmap.hbs            |   30 +
 nifi-site/src/pages/html/screencasts.hbs        |   50 +
 nifi-site/src/pages/html/user-guide.hbs         |    7 +
 nifi-site/src/pages/markdown/quickstart.md      |  101 +
 nifi-site/src/pages/markdown/release-guide.md   |  298 +++
 nifi-site/src/scss/_settings.scss               | 1465 ++++++++++++
 nifi-site/src/scss/app.scss                     |  207 ++
 nifi/LICENSE                                    |  343 +--
 nifi/NOTICE                                     |    8 -
 nifi/README.md                                  |   40 +-
 nifi/nifi-api/pom.xml                           |    4 +-
 nifi/nifi-assembly/LICENSE                      |  803 +++++++
 nifi/nifi-assembly/NOTICE                       |  529 +++++
 nifi/nifi-assembly/README.md                    |  124 +
 nifi/nifi-assembly/pom.xml                      |   13 +-
 .../src/main/assembly/dependencies.xml          |    6 +-
 nifi/nifi-bootstrap/pom.xml                     |    2 +-
 .../nifi-data-provenance-utils/pom.xml          |    4 +-
 .../nifi-expression-language/pom.xml            |    4 +-
 .../nifi-commons/nifi-flowfile-packager/pom.xml |    4 +-
 nifi/nifi-commons/nifi-logging-utils/pom.xml    |    4 +-
 .../nifi-processor-utilities/pom.xml            |    4 +-
 nifi/nifi-commons/nifi-properties/pom.xml       |    4 +-
 nifi/nifi-commons/nifi-security-utils/pom.xml   |    4 +-
 nifi/nifi-commons/nifi-socket-utils/pom.xml     |    4 +-
 nifi/nifi-commons/nifi-utils/pom.xml            |    4 +-
 nifi/nifi-commons/nifi-web-utils/pom.xml        |    4 +-
 nifi/nifi-commons/nifi-write-ahead-log/pom.xml  |    4 +-
 nifi/nifi-commons/pom.xml                       |    5 +-
 nifi/nifi-docs/LICENSE                          |  235 ++
 nifi/nifi-docs/NOTICE                           |    5 +
 nifi/nifi-docs/pom.xml                          |   36 +-
 .../src/main/asciidoc/asciidoc-mod.css          |  418 ++++
 .../src/main/asciidoc/developer-guide.adoc      | 2126 +++++++++++++++++-
 .../src/main/assembly/dependencies.xml          |   16 +
 .../nifi-processor-bundle-archetype/pom.xml     |    2 +-
 .../src/main/java/MyProcessor.java              |   10 +-
 nifi/nifi-maven-archetypes/pom.xml              |    2 +-
 nifi/nifi-mock/pom.xml                          |    4 +-
 .../nifi/util/StandardProcessorTestRunner.java  |    7 +-
 .../util/TestStandardProcessorTestRunner.java   |    2 +
 .../nifi-framework-nar/pom.xml                  |    4 +-
 .../src/main/resources/META-INF/LICENSE         |  264 +++
 .../src/main/resources/META-INF/NOTICE          |  131 ++
 .../nifi-framework/nifi-administration/pom.xml  |    4 +-
 .../nifi-framework/nifi-client-dto/pom.xml      |    2 +-
 .../nifi-cluster-authorization-provider/pom.xml |    2 +-
 .../nifi-cluster-protocol/pom.xml               |    2 +-
 .../nifi-framework/nifi-cluster-web/pom.xml     |    2 +-
 .../nifi-framework/nifi-cluster/pom.xml         |    2 +-
 .../nifi-file-authorization-provider/pom.xml    |    4 +-
 .../nifi-framework-core-api/pom.xml             |    4 +-
 .../nifi-framework/nifi-framework-core/pom.xml  |    2 +-
 .../nifi-framework/nifi-nar-utils/pom.xml       |    2 +-
 .../nifi-framework/nifi-resources/pom.xml       |    2 +-
 .../src/main/resources/conf/logback.xml         |    4 +-
 .../nifi-framework/nifi-runtime/pom.xml         |    2 +-
 .../nifi-framework/nifi-security/pom.xml        |    4 +-
 .../nifi-framework/nifi-site-to-site/pom.xml    |    4 +-
 .../nifi-framework/nifi-user-actions/pom.xml    |    4 +-
 .../nifi-web/nifi-custom-ui-utilities/pom.xml   |    2 +-
 .../nifi-framework/nifi-web/nifi-jetty/pom.xml  |    2 +-
 .../nifi-web/nifi-web-api/pom.xml               |    6 +-
 .../nifi/web/controller/ControllerFacade.java   |   15 +-
 .../src/main/resources/META-INF/NOTICE          |   27 +
 .../nifi-web/nifi-web-docs/pom.xml              |    2 +-
 .../src/main/resources/META-INF/NOTICE          |   19 +
 .../nifi-web/nifi-web-error/pom.xml             |    2 +-
 .../nifi-web-optimistic-locking/pom.xml         |    2 +-
 .../nifi-web/nifi-web-security/pom.xml          |    2 +-
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml |   16 +-
 .../org/apache/nifi/web/servlet/ConvertSvg.java |  117 -
 .../apache/nifi/web/servlet/DownloadSvg.java    |   89 +
 .../src/main/resources/META-INF/LICENSE         |  434 ++++
 .../nifi-web-ui/src/main/webapp/WEB-INF/web.xml |   30 +-
 .../src/main/webapp/css/provenance.css          |    5 +
 .../src/main/webapp/js/nf/canvas/nf-actions.js  |    2 -
 .../webapp/js/nf/canvas/nf-canvas-toolbox.js    |   58 +-
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js |   19 +-
 .../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 +-
 .../webapp/js/nf/canvas/nf-process-group.js     |    6 +-
 .../nf-remote-process-group-configuration.js    |    3 +
 .../js/nf/canvas/nf-remote-process-group.js     |    2 +-
 .../webapp/js/nf/cluster/nf-cluster-table.js    |  319 ++-
 .../webapp/js/nf/counters/nf-counters-table.js  |   63 +-
 .../webapp/js/nf/history/nf-history-table.js    |  147 +-
 .../src/main/webapp/js/nf/nf-common.js          |    8 +-
 .../js/nf/provenance/nf-provenance-lineage.js   |   17 +-
 .../js/nf/provenance/nf-provenance-table.js     |  103 +-
 .../webapp/js/nf/summary/nf-summary-table.js    |  523 ++---
 .../src/main/webapp/js/nf/summary/nf-summary.js |    6 -
 .../js/nf/templates/nf-templates-table.js       |   73 +-
 .../main/webapp/js/nf/users/nf-users-table.js   |  440 ++--
 .../nifi-framework/nifi-web/pom.xml             |   10 +-
 .../nifi-framework/pom.xml                      |    4 +-
 .../nifi-framework-bundle/pom.xml               |   36 +-
 .../nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml  |    4 +-
 .../src/main/resources/META-INF/NOTICE          |   35 +
 .../nifi-hdfs-processors/pom.xml                |    2 +-
 .../nifi-nar-bundles/nifi-hadoop-bundle/pom.xml |    6 +-
 .../nifi-hadoop-libraries-nar/pom.xml           |    4 +-
 .../src/main/resources/META-INF/LICENSE         |  359 +++
 .../src/main/resources/META-INF/NOTICE          |  237 ++
 .../nifi-hadoop-libraries-bundle/pom.xml        |    4 +-
 nifi/nifi-nar-bundles/nifi-jetty-bundle/pom.xml |    4 +-
 .../src/main/resources/META-INF/NOTICE          |   37 +
 .../nifi-kafka-bundle/nifi-kafka-nar/pom.xml    |    4 +-
 .../src/main/resources/META-INF/LICENSE         |  299 +++
 .../src/main/resources/META-INF/NOTICE          |   72 +
 .../nifi-kafka-processors/pom.xml               |    2 +-
 nifi/nifi-nar-bundles/nifi-kafka-bundle/pom.xml |    4 +-
 .../pom.xml                                     |    2 +-
 .../nifi-provenance-repository-nar/pom.xml      |    4 +-
 .../src/main/resources/META-INF/NOTICE          |  202 ++
 .../nifi-volatile-provenance-repository/pom.xml |    2 +-
 .../nifi-provenance-repository-bundle/pom.xml   |    8 +-
 .../nifi-standard-nar/pom.xml                   |    4 +-
 .../src/main/resources/META-INF/LICENSE         |  292 +++
 .../src/main/resources/META-INF/NOTICE          |  119 +
 .../nifi-standard-prioritizers/pom.xml          |    2 +-
 .../nifi-standard-processors/pom.xml            |    2 +-
 .../nifi/processors/standard/BinFiles.java      |  351 +++
 .../apache/nifi/processors/standard/GetFTP.java |    5 +
 .../nifi/processors/standard/JmsConsumer.java   |  143 +-
 .../nifi/processors/standard/MergeContent.java  |  403 +---
 .../nifi/processors/standard/PostHTTP.java      |   30 +-
 .../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/CaptureServlet.java     |   54 +
 .../processors/standard/TestJmsConsumer.java    |  173 ++
 .../nifi/processors/standard/TestPostHTTP.java  |  249 ++
 .../nifi/processors/standard/TestServer.java    |   27 +-
 .../nifi-standard-reporting-tasks/pom.xml       |    2 +-
 .../nifi-standard-bundle/pom.xml                |   10 +-
 .../pom.xml                                     |    2 +-
 .../pom.xml                                     |    2 +-
 .../nifi-distributed-cache-protocol/pom.xml     |    2 +-
 .../nifi-distributed-cache-server/pom.xml       |    2 +-
 .../nifi-distributed-cache-services-nar/pom.xml |    4 +-
 .../src/main/resources/META-INF/NOTICE          |   19 +
 .../pom.xml                                     |    4 +-
 .../nifi-load-distribution-service-api/pom.xml  |    4 +-
 .../nifi-ssl-context-nar/pom.xml                |    4 +-
 .../src/main/resources/META-INF/NOTICE          |   19 +
 .../nifi-ssl-context-service/pom.xml            |    2 +-
 .../nifi-ssl-context-bundle/pom.xml             |    4 +-
 .../nifi-ssl-context-service-api/pom.xml        |    2 +-
 .../nifi-standard-services-api-nar/pom.xml      |    4 +-
 .../nifi-standard-services/pom.xml              |    4 +-
 .../nifi-update-attribute-model/pom.xml         |    2 +-
 .../nifi-update-attribute-nar/pom.xml           |    4 +-
 .../src/main/resources/META-INF/NOTICE          |   19 +
 .../nifi-update-attribute-processor/pom.xml     |    2 +-
 .../nifi-update-attribute-ui/pom.xml            |    2 +-
 .../update/attributes/api/RuleResource.java     |   16 +-
 .../src/main/resources/META-INF/LICENSE         |  238 ++
 .../src/main/resources/META-INF/NOTICE          |   49 +
 .../nifi-update-attribute-bundle/pom.xml        |   10 +-
 nifi/nifi-nar-bundles/pom.xml                   |   31 +-
 nifi/pom.xml                                    |   90 +-
 199 files changed, 13447 insertions(+), 2075 deletions(-)
----------------------------------------------------------------------



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

Posted by mc...@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/NIFI-353
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(-)
----------------------------------------------------------------------



[18/51] [abbrv] incubator-nifi git commit: NIFI-333 - Adding failure test for DecompressContent

Posted by mc...@apache.org.
NIFI-333 - Adding failure test for DecompressContent

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

Branch: refs/heads/NIFI-353
Commit: 361ac1f1e5c8ee943220b337c92cd49616a02194
Parents: 0f8d00d
Author: danbress <db...@onyxconsults.com>
Authored: Sun Feb 15 09:08:22 2015 -0500
Committer: danbress <db...@onyxconsults.com>
Committed: Sun Feb 15 09:08:22 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/TestCompressContent.java  | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/361ac1f1/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java
index 71c8583..df1d506 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestCompressContent.java
@@ -109,4 +109,22 @@ public class TestCompressContent {
         flowFile.assertAttributeEquals("filename", "SampleFile.txt.gz");
 
     }
+    
+    @Test
+    public void testDecompressFailure() throws IOException {
+    	final TestRunner runner = TestRunners.newTestRunner(CompressContent.class);
+        runner.setProperty(CompressContent.MODE, "decompress");
+        runner.setProperty(CompressContent.COMPRESSION_FORMAT, "gzip");
+        
+        byte[] data = new byte[]{1,2,3,4,5,6,7,8,9,10};
+        runner.enqueue(data);
+        
+        
+        assertTrue(runner.setProperty(CompressContent.UPDATE_FILENAME, "true").isValid());
+        runner.run();
+        runner.assertQueueEmpty();
+        runner.assertAllFlowFilesTransferred(CompressContent.REL_FAILURE, 1);
+        
+        runner.getFlowFilesForRelationship(CompressContent.REL_FAILURE).get(0).assertContentEquals(data);
+    }
 }


[25/51] [abbrv] incubator-nifi git commit: NIFI-333 - Removing exception handling in SegmentContent From Mark - 'Theres no exception that could get thrown in there unless theres something weird - in which case the framework should catch it and handle it'

Posted by mc...@apache.org.
NIFI-333 - Removing exception handling in SegmentContent
>From Mark - 'Theres no exception that could get thrown in there unless
theres something weird - in which case the framework should catch it and
handle 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/c8810c04
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c8810c04
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c8810c04

Branch: refs/heads/NIFI-353
Commit: c8810c04d8dcb6823c7e59e85158a0c3cb18675f
Parents: 0fa1b16
Author: danbress <db...@onyxconsults.com>
Authored: Tue Feb 17 11:17:46 2015 -0500
Committer: danbress <db...@onyxconsults.com>
Committed: Tue Feb 17 11:17:46 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/SegmentContent.java     | 114 +++++++++----------
 1 file changed, 54 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c8810c04/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
index dfdd401..cf0539e 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java
@@ -25,6 +25,11 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -34,12 +39,6 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.behavior.EventDriven;
-import org.apache.nifi.annotation.behavior.SideEffectFree;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 
 @EventDriven
@@ -102,62 +101,57 @@ public class SegmentContent extends AbstractProcessor {
             return;
         }
 
-        try {
-            final String segmentId = UUID.randomUUID().toString();
-            final long segmentSize = context.getProperty(SIZE).asDataSize(DataUnit.B).longValue();
-
-            final String originalFileName = flowFile.getAttribute(CoreAttributes.FILENAME.key());
-
-            if (flowFile.getSize() <= segmentSize) {
-                flowFile = session.putAttribute(flowFile, SEGMENT_ID, segmentId);
-                flowFile = session.putAttribute(flowFile, SEGMENT_INDEX, "1");
-                flowFile = session.putAttribute(flowFile, SEGMENT_COUNT, "1");
-                flowFile = session.putAttribute(flowFile, SEGMENT_ORIGINAL_FILENAME, originalFileName);
-
-                flowFile = session.putAttribute(flowFile, FRAGMENT_ID, segmentId);
-                flowFile = session.putAttribute(flowFile, FRAGMENT_INDEX, "1");
-                flowFile = session.putAttribute(flowFile, FRAGMENT_COUNT, "1");
-
-                FlowFile clone = session.clone(flowFile);
-                session.transfer(flowFile, REL_ORIGINAL);
-                session.transfer(clone, REL_SEGMENTS);
-                return;
-            }
-
-            int totalSegments = (int) (flowFile.getSize() / segmentSize);
-            if (totalSegments * segmentSize < flowFile.getSize()) {
-                totalSegments++;
-            }
-
-            final Map<String, String> segmentAttributes = new HashMap<>();
-            segmentAttributes.put(SEGMENT_ID, segmentId);
-            segmentAttributes.put(SEGMENT_COUNT, String.valueOf(totalSegments));
-            segmentAttributes.put(SEGMENT_ORIGINAL_FILENAME, originalFileName);
-
-            segmentAttributes.put(FRAGMENT_ID, segmentId);
-            segmentAttributes.put(FRAGMENT_COUNT, String.valueOf(totalSegments));
-
-            final Set<FlowFile> segmentSet = new HashSet<>();
-            for (int i = 1; i <= totalSegments; i++) {
-                final long segmentOffset = segmentSize * (i - 1);
-                FlowFile segment = session.clone(flowFile, segmentOffset, Math.min(segmentSize, flowFile.getSize() - segmentOffset));
-                segmentAttributes.put(SEGMENT_INDEX, String.valueOf(i));
-                segmentAttributes.put(FRAGMENT_INDEX, String.valueOf(i));
-                segment = session.putAllAttributes(segment, segmentAttributes);
-                segmentSet.add(segment);
-            }
-
-            session.transfer(segmentSet, REL_SEGMENTS);
+        final String segmentId = UUID.randomUUID().toString();
+        final long segmentSize = context.getProperty(SIZE).asDataSize(DataUnit.B).longValue();
+
+        final String originalFileName = flowFile.getAttribute(CoreAttributes.FILENAME.key());
+
+        if (flowFile.getSize() <= segmentSize) {
+            flowFile = session.putAttribute(flowFile, SEGMENT_ID, segmentId);
+            flowFile = session.putAttribute(flowFile, SEGMENT_INDEX, "1");
+            flowFile = session.putAttribute(flowFile, SEGMENT_COUNT, "1");
+            flowFile = session.putAttribute(flowFile, SEGMENT_ORIGINAL_FILENAME, originalFileName);
+
+            flowFile = session.putAttribute(flowFile, FRAGMENT_ID, segmentId);
+            flowFile = session.putAttribute(flowFile, FRAGMENT_INDEX, "1");
+            flowFile = session.putAttribute(flowFile, FRAGMENT_COUNT, "1");
+
+            FlowFile clone = session.clone(flowFile);
             session.transfer(flowFile, REL_ORIGINAL);
+            session.transfer(clone, REL_SEGMENTS);
+            return;
+        }
 
-            if (totalSegments <= 10) {
-                getLogger().info("Segmented {} into {} segments: {}", new Object[]{flowFile, totalSegments, segmentSet});
-            } else {
-                getLogger().info("Segmented {} into {} segments", new Object[]{flowFile, totalSegments});
-            }
-        } catch (final Exception e) {
-            throw new ProcessException(e);
+        int totalSegments = (int) (flowFile.getSize() / segmentSize);
+        if (totalSegments * segmentSize < flowFile.getSize()) {
+            totalSegments++;
         }
-    }
 
+        final Map<String, String> segmentAttributes = new HashMap<>();
+        segmentAttributes.put(SEGMENT_ID, segmentId);
+        segmentAttributes.put(SEGMENT_COUNT, String.valueOf(totalSegments));
+        segmentAttributes.put(SEGMENT_ORIGINAL_FILENAME, originalFileName);
+
+        segmentAttributes.put(FRAGMENT_ID, segmentId);
+        segmentAttributes.put(FRAGMENT_COUNT, String.valueOf(totalSegments));
+
+        final Set<FlowFile> segmentSet = new HashSet<>();
+        for (int i = 1; i <= totalSegments; i++) {
+            final long segmentOffset = segmentSize * (i - 1);
+            FlowFile segment = session.clone(flowFile, segmentOffset, Math.min(segmentSize, flowFile.getSize() - segmentOffset));
+            segmentAttributes.put(SEGMENT_INDEX, String.valueOf(i));
+            segmentAttributes.put(FRAGMENT_INDEX, String.valueOf(i));
+            segment = session.putAllAttributes(segment, segmentAttributes);
+            segmentSet.add(segment);
+        }
+
+        session.transfer(segmentSet, REL_SEGMENTS);
+        session.transfer(flowFile, REL_ORIGINAL);
+
+        if (totalSegments <= 10) {
+            getLogger().info("Segmented {} into {} segments: {}", new Object[]{flowFile, totalSegments, segmentSet});
+        } else {
+            getLogger().info("Segmented {} into {} segments", new Object[]{flowFile, totalSegments});
+        }
+    }
 }


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

Posted by mc...@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/NIFI-353
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'}},


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

Posted by mc...@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;


[30/51] [abbrv] incubator-nifi git commit: NIFI-344 improving error message, and setting a placeHolder for the field

Posted by mc...@apache.org.
NIFI-344 improving error message, and setting a placeHolder for the
field

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

Branch: refs/heads/NIFI-353
Commit: 750c4ebb6c1d2f91e408ae7d8cbaf252cbcef30a
Parents: 10b5b3c
Author: danbress <db...@onyxconsults.com>
Authored: Thu Feb 19 18:15:29 2015 -0500
Committer: danbress <db...@onyxconsults.com>
Committed: Thu Feb 19 18:54:44 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/web/api/RemoteProcessGroupResource.java | 4 ++--
 .../WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp  | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/750c4ebb/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
index 3ddebd1..27fa292 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
@@ -372,7 +372,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
         }
         
         if (!(uri.getScheme().equalsIgnoreCase("http") || uri.getScheme().equalsIgnoreCase("https"))) {
-            throw new IllegalArgumentException("The specified remote process group URL is invalid, only http and https are supported: " + requestProcessGroupDTO.getTargetUri());
+            throw new IllegalArgumentException("The specified remote process group URL is invalid because it is not http or https: " + requestProcessGroupDTO.getTargetUri());
         }
 
         // normalize the uri to the other controller
@@ -866,7 +866,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
             }
             
             if (!(uri.getScheme().equalsIgnoreCase("http") || uri.getScheme().equalsIgnoreCase("https"))) {
-                throw new IllegalArgumentException("The specified remote process group URL is invalid, only http and https are supported: " + requestRemoteProcessGroup.getTargetUri());
+                throw new IllegalArgumentException("The specified remote process group URL is invalid because it is not http or https: " + requestRemoteProcessGroup.getTargetUri());
             }
 
             // normalize the uri to the other controller

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/750c4ebb/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
index 1c230da..a479384 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
@@ -20,7 +20,7 @@
         <div class="setting">
             <div class="setting-name">URL</div>
             <div class="setting-field">
-                <input id="new-remote-process-group-uri" type="text" placeholder="https://"/>
+                <input id="new-remote-process-group-uri" type="text" placeholder="https://remotehost:8080/nifi"/>
             </div>
         </div>
     </div>


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

Posted by mc...@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/NIFI-353
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);
                             	


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

Posted by mc...@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/NIFI-353
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;
+    }
+}


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

Posted by mc...@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/4ac6dfd5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/4ac6dfd5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/4ac6dfd5

Branch: refs/heads/NIFI-353
Commit: 4ac6dfd5e5121291f11e2180072eb3bd5ff6d0a7
Parents: c5f4dff d8954ab
Author: Jenn Barnabee <je...@gmail.com>
Authored: Tue Feb 24 13:45:49 2015 -0500
Committer: Jenn Barnabee <je...@gmail.com>
Committed: Tue Feb 24 13:45:49 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 ++
 .../nifi/remote/io/CompressionInputStream.java  |   2 +-
 .../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/controller/StandardFunnel.java  |   2 +-
 .../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 +-
 .../scheduling/QuartzSchedulingAgent.java       |  21 +-
 .../controller/scheduling/ScheduleState.java    |  18 +-
 .../scheduling/TimerDrivenSchedulingAgent.java  | 115 ++-
 .../tasks/ContinuallyRunConnectableTask.java    |  32 +-
 .../tasks/ContinuallyRunProcessorTask.java      |  32 +-
 .../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 --
 .../web/api/RemoteProcessGroupResource.java     |   8 +
 .../canvas/new-remote-process-group-dialog.jsp  |   2 +-
 .../src/main/webapp/js/nf/canvas/nf-actions.js  |  26 +-
 .../main/webapp/js/nf/canvas/nf-context-menu.js |   2 +-
 .../hadoop/CreateHadoopSequenceFile.java        |   4 +-
 .../standard/Base64EncodeContent.java           |  13 +-
 .../processors/standard/CompressContent.java    |  27 +-
 .../nifi/processors/standard/HashContent.java   |  14 +-
 .../nifi/processors/standard/PutEmail.java      |   5 +-
 .../processors/standard/SegmentContent.java     | 114 ++-
 .../nifi/processors/standard/TransformXml.java  |   3 +-
 .../standard/TestCompressContent.java           |  18 +
 .../nifi/processors/standard/TestPutEmail.java  |  45 +
 nifi/pom.xml                                    |   6 +
 160 files changed, 7186 insertions(+), 3545 deletions(-)
----------------------------------------------------------------------



[44/51] [abbrv] incubator-nifi git commit: NIFI-366: removed logic to check for invalid ports and throw exception if all ports are invalid. Instead, we will now just run but will be unable to transfer any data; this is preferred anyone because we should

Posted by mc...@apache.org.
NIFI-366: removed logic to check for invalid ports and throw exception if all ports are invalid. Instead, we will now just run but will be unable to transfer any data; this is preferred anyone because we should still allow the state to be set to transmit, so that when the ports are no longer invalid we can start transmitting immediately.


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

Branch: refs/heads/NIFI-353
Commit: e370d7d7e3a78acdda5f7a3bc6d3c0a87768f7fa
Parents: d8954ab
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 24 14:03:00 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 24 14:03:00 2015 -0500

----------------------------------------------------------------------
 .../nifi/remote/StandardRemoteProcessGroup.java | 44 --------------------
 1 file changed, 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e370d7d7/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 6b70fe6..55575c6 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
@@ -915,50 +915,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         try {
             verifyCanStartTransmitting();
 
-            // Check if any port is invalid
-            boolean invalidPort = false;
-            for (final Port port : getInputPorts()) {
-                if (!port.isValid()) {
-                    invalidPort = true;
-                    break;
-                }
-            }
-
-            if (!invalidPort) {
-                for (final Port port : getOutputPorts()) {
-                    if (!port.isValid()) {
-                        invalidPort = true;
-                    }
-                }
-            }
-
-            // if any port is invalid, refresh contents to check if it is still invalid
-            boolean allPortsInvalid = invalidPort;
-            if (invalidPort) {
-                try {
-                    refreshFlowContents();
-                } catch (final CommunicationsException e) {
-                    logger.warn("{} Attempted to refresh Flow Contents because at least one port is invalid but failed due to {}", this, e);
-                }
-
-                for (final Port port : getInputPorts()) {
-                    if (port.isValid()) {
-                        allPortsInvalid = false;
-                        break;
-                    }
-                }
-                for (final Port port : getOutputPorts()) {
-                    if (port.isValid()) {
-                        allPortsInvalid = false;
-                        break;
-                    }
-                }
-            }
-
-            if (allPortsInvalid) {
-                throw new IllegalStateException("Cannot Enable Transmission because all Input Ports & Output Ports to this Remote Process Group are in invalid states");
-            }
-
             for (final Port port : getInputPorts()) {
                 // if port is not valid, don't start it because it will never become valid.
                 // Validation is based on connections and whether or not the remote target exists.


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

Posted by mc...@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/NIFI-353
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);
+                                }
+                            }
                         }
                     }
                 }


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

Posted by mc...@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/NIFI-353
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();
+        }
     }
     
 }


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

Posted by mc...@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/NIFI-353
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


[23/51] [abbrv] incubator-nifi git commit: NIFI-282: Code cleanup

Posted by mc...@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/NIFI-353
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:


[28/51] [abbrv] incubator-nifi git commit: NIFI-344 - Defaulting new Remote Process Group value

Posted by mc...@apache.org.
NIFI-344 - Defaulting new Remote Process Group value

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

Branch: refs/heads/NIFI-353
Commit: 10b5b3c90702115698b9d88180a806a44e96dda1
Parents: 0909082
Author: danbress <db...@onyxconsults.com>
Authored: Wed Feb 18 21:55:47 2015 -0500
Committer: danbress <db...@onyxconsults.com>
Committed: Wed Feb 18 21:55:47 2015 -0500

----------------------------------------------------------------------
 .../WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/10b5b3c9/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
index d682c39..1c230da 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-remote-process-group-dialog.jsp
@@ -20,7 +20,7 @@
         <div class="setting">
             <div class="setting-name">URL</div>
             <div class="setting-field">
-                <input id="new-remote-process-group-uri" type="text"/>
+                <input id="new-remote-process-group-uri" type="text" placeholder="https://"/>
             </div>
         </div>
     </div>


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

Posted by mc...@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/50744bfd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/50744bfd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/50744bfd

Branch: refs/heads/NIFI-353
Commit: 50744bfdc608c2395b7af281a587d241356e699c
Parents: 97aa53d 4ac6dfd
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 25 14:35:45 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 25 14:35:45 2015 -0500

----------------------------------------------------------------------
 .../src/main/asciidoc/administration-guide.adoc | 43 ++++++++++++++------
 nifi/nifi-docs/src/main/asciidoc/overview.adoc  |  4 +-
 .../nifi-docs/src/main/asciidoc/user-guide.adoc | 11 +++++
 3 files changed, 44 insertions(+), 14 deletions(-)
----------------------------------------------------------------------



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

Posted by mc...@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/b6f40264
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b6f40264
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b6f40264

Branch: refs/heads/NIFI-353
Commit: b6f40264e8c5c8ec7a15597c7886538f82748a99
Parents: 1c8d005 d76fe22
Author: Jenn Barnabee <je...@gmail.com>
Authored: Fri Feb 20 10:41:06 2015 -0500
Committer: Jenn Barnabee <je...@gmail.com>
Committed: Fri Feb 20 10:41:06 2015 -0500

----------------------------------------------------------------------
 README.md                                       |   2 +-
 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 -
 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 -
 .../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  |  46 ++++++--
 .../webapp/js/nf/canvas/nf-canvas-header.js     | 114 ++++++++++++-------
 .../webapp/js/nf/canvas/nf-canvas-toolbar.js    |  18 +--
 .../main/webapp/js/nf/canvas/nf-canvas-utils.js |  25 ++++
 .../main/webapp/js/nf/canvas/nf-context-menu.js |   7 +-
 .../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 +-
 .../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 -
 60 files changed, 170 insertions(+), 145 deletions(-)
----------------------------------------------------------------------



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

Posted by mc...@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();
+    }
+}


[38/51] [abbrv] incubator-nifi git commit: Merge branch 'NIFI-344' of https://github.com/danbress/incubator-nifi into develop

Posted by mc...@apache.org.
Merge branch 'NIFI-344' of https://github.com/danbress/incubator-nifi into develop

This closes #29


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

Branch: refs/heads/NIFI-353
Commit: 42f69196c2e9afff18b864a054a5e6e30d5fe351
Parents: 57b5d58 750c4eb
Author: Matt Gilman <ma...@gmail.com>
Authored: Mon Feb 23 07:24:39 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Feb 23 07:24:39 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(-)
----------------------------------------------------------------------



[46/51] [abbrv] incubator-nifi git commit: NIFI-382: Made nifi.bored.yield.duration property

Posted by mc...@apache.org.
NIFI-382: Made nifi.bored.yield.duration property


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

Branch: refs/heads/NIFI-353
Commit: 5aef55b5f973a290a70aa3607dbcbba47b03fe79
Parents: a956623
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 25 12:01:31 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 25 12:01:31 2015 -0500

----------------------------------------------------------------------
 nifi/nifi-assembly/pom.xml                            |  1 +
 .../java/org/apache/nifi/util/NiFiProperties.java     |  5 +++++
 .../scheduling/TimerDrivenSchedulingAgent.java        | 14 +++++++++++---
 .../src/main/resources/conf/nifi.properties           |  2 ++
 4 files changed, 19 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5aef55b5/nifi/nifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml
index 8d16d5e..9a55cbc 100644
--- a/nifi/nifi-assembly/pom.xml
+++ b/nifi/nifi-assembly/pom.xml
@@ -173,6 +173,7 @@
         <nifi.flowcontroller.graceful.shutdown.period>10 sec</nifi.flowcontroller.graceful.shutdown.period>
         <nifi.flowservice.writedelay.interval>500 ms</nifi.flowservice.writedelay.interval>
         <nifi.administrative.yield.duration>30 sec</nifi.administrative.yield.duration>
+        <nifi.bored.yield.duration>10 millis</nifi.bored.yield.duration>
 
         <nifi.flow.configuration.file>./conf/flow.xml.gz</nifi.flow.configuration.file>
         <nifi.flow.configuration.archive.dir>./conf/archive/</nifi.flow.configuration.archive.dir>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5aef55b5/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index 10e348d..87a82d4 100644
--- a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -63,6 +63,7 @@ public class NiFiProperties extends Properties {
     public static final String TEMPLATE_DIRECTORY = "nifi.templates.directory";
     public static final String ADMINISTRATIVE_YIELD_DURATION = "nifi.administrative.yield.duration";
     public static final String PERSISTENT_STATE_DIRECTORY = "nifi.persistent.state.directory";
+    public static final String BORED_YIELD_DURATION = "nifi.bored.yield.duration";
 
     // content repository properties
     public static final String REPOSITORY_CONTENT_PREFIX = "nifi.content.repository.directory.";
@@ -200,6 +201,7 @@ public class NiFiProperties extends Properties {
     public static final String DEFAULT_ADMINISTRATIVE_YIELD_DURATION = "30 sec";
     public static final String DEFAULT_PERSISTENT_STATE_DIRECTORY = "./conf/state";
     public static final String DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY = "5 mins";
+    public static final String DEFAULT_BORED_YIELD_DURATION = "10 millis";
 
     // cluster common defaults
     public static final String DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL = "5 sec";
@@ -873,4 +875,7 @@ public class NiFiProperties extends Properties {
         return value;
     }
 
+    public String getBoredYieldDuration() {
+        return getProperty(BORED_YIELD_DURATION, DEFAULT_BORED_YIELD_DURATION);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5aef55b5/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 17fb9f8..f3eecbd 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
@@ -37,12 +37,13 @@ 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.apache.nifi.util.NiFiProperties;
 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 long noWorkYieldNanos;
     
     private final FlowController flowController;
     private final FlowEngine flowEngine;
@@ -56,6 +57,13 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
         this.flowEngine = flowEngine;
         this.contextFactory = contextFactory;
         this.encryptor = encryptor;
+        
+        final String boredYieldDuration = NiFiProperties.getInstance().getBoredYieldDuration();
+        try {
+            noWorkYieldNanos = FormatUtils.getTimeDuration(boredYieldDuration, TimeUnit.NANOSECONDS);
+        } catch (final IllegalArgumentException e) {
+            throw new RuntimeException("Failed to create SchedulingAgent because the " + NiFiProperties.BORED_YIELD_DURATION + " property is set to an invalid time duration: " + boredYieldDuration);
+        }
     }
 
     @Override
@@ -141,7 +149,7 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
                                 }
                             }
                         }
-                    } else if ( shouldYield ) {
+                    } else if ( noWorkYieldNanos > 0L && 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();
@@ -155,7 +163,7 @@ public class TimerDrivenSchedulingAgent implements SchedulingAgent {
                         if (scheduledFuture.cancel(false)) {
                             synchronized (scheduleState) {
                                 if ( scheduleState.isScheduled() ) {
-                                    final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, NO_WORK_YIELD_NANOS, 
+                                    final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, noWorkYieldNanos, 
                                             connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
                                     
                                     scheduleState.replaceFuture(scheduledFuture, newFuture);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5aef55b5/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index fa55623..fd16cb5 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -21,6 +21,8 @@ nifi.flowcontroller.autoResumeState=${nifi.flowcontroller.autoResumeState}
 nifi.flowcontroller.graceful.shutdown.period=${nifi.flowcontroller.graceful.shutdown.period}
 nifi.flowservice.writedelay.interval=${nifi.flowservice.writedelay.interval}
 nifi.administrative.yield.duration=${nifi.administrative.yield.duration}
+# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
+nifi.bored.yield.duration=${nifi.bored.yield.duration}
 
 nifi.authority.provider.configuration.file=${nifi.authority.provider.configuration.file}
 nifi.reporting.task.configuration.file=${nifi.reporting.task.configuration.file}


[17/51] [abbrv] incubator-nifi git commit: NIFI-333 - catching ProcessException instead of Exception when appropriate catching ProcessException in processors instead of exception

Posted by mc...@apache.org.
NIFI-333 - catching ProcessException instead of Exception
when appropriate catching ProcessException in processors instead of
exception

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

Branch: refs/heads/NIFI-353
Commit: 0f8d00d5fff947647926570ccfa4b83ac9411e4b
Parents: b8ade5b
Author: danbress <db...@onyxconsults.com>
Authored: Sat Feb 14 12:20:18 2015 -0500
Committer: danbress <db...@onyxconsults.com>
Committed: Sat Feb 14 12:28:33 2015 -0500

----------------------------------------------------------------------
 .../hadoop/CreateHadoopSequenceFile.java        |  4 +--
 .../standard/Base64EncodeContent.java           | 13 +++++-----
 .../processors/standard/CompressContent.java    | 27 ++++++++++----------
 .../nifi/processors/standard/HashContent.java   | 14 +++++-----
 .../nifi/processors/standard/PutEmail.java      |  5 ++--
 .../nifi/processors/standard/TransformXml.java  |  3 ++-
 6 files changed, 35 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0f8d00d5/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
index 1422a7b..a031923 100644
--- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
+++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java
@@ -22,6 +22,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
@@ -33,7 +34,6 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processors.hadoop.util.SequenceFileWriter;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
 
 /**
  * <p>
@@ -167,7 +167,7 @@ public class CreateHadoopSequenceFile extends AbstractHadoopProcessor {
             flowFile = sequenceFileWriter.writeSequenceFile(flowFile, session, hdfsResources.get().getKey(), compressionType);
             session.transfer(flowFile, RELATIONSHIP_SUCCESS);
             getLogger().info("Transferred flowfile {} to {}", new Object[]{flowFile, RELATIONSHIP_SUCCESS});
-        } catch (Exception e) {
+        } catch (ProcessException e) {
             getLogger().error("Failed to create Sequence File. Transferring {} to 'failure'", new Object[]{flowFile}, e);
             session.transfer(flowFile, RELATIONSHIP_FAILURE);
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0f8d00d5/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java
index d9175e0..cd272ff 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java
@@ -28,6 +28,11 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.codec.binary.Base64InputStream;
 import org.apache.commons.codec.binary.Base64OutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.logging.ProcessorLog;
@@ -36,11 +41,7 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.behavior.EventDriven;
-import org.apache.nifi.annotation.behavior.SideEffectFree;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.processors.standard.util.ValidatingBase64InputStream;
 import org.apache.nifi.util.StopWatch;
@@ -136,7 +137,7 @@ public class Base64EncodeContent extends AbstractProcessor {
             logger.info("Successfully {} {}", new Object[]{encode ? "encoded" : "decoded", flowFile});
             session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
             session.transfer(flowFile, REL_SUCCESS);
-        } catch (Exception e) {
+        } catch (ProcessException e) {
             logger.error("Failed to {} {} due to {}", new Object[]{encode ? "encode" : "decode", flowFile, e});
             session.transfer(flowFile, REL_FAILURE);
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0f8d00d5/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
index cf20f16..e631cd0 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java
@@ -32,30 +32,31 @@ import java.util.concurrent.TimeUnit;
 import lzma.sdk.lzma.Decoder;
 import lzma.streams.LzmaInputStream;
 import lzma.streams.LzmaOutputStream;
+
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.stream.io.BufferedInputStream;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.GZIPOutputStream;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.behavior.EventDriven;
-import org.apache.nifi.annotation.behavior.SideEffectFree;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.GZIPOutputStream;
 import org.apache.nifi.util.ObjectHolder;
 import org.apache.nifi.util.StopWatch;
-
-import org.apache.commons.compress.compressors.CompressorStreamFactory;
-import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
-import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
 import org.tukaani.xz.LZMA2Options;
 import org.tukaani.xz.XZInputStream;
 import org.tukaani.xz.XZOutputStream;
@@ -290,7 +291,7 @@ public class CompressContent extends AbstractProcessor {
                 compressionMode.toLowerCase(), flowFile, compressionFormat, sizeBeforeCompression, sizeAfterCompression});
             session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getDuration(TimeUnit.MILLISECONDS));
             session.transfer(flowFile, REL_SUCCESS);
-        } catch (final Exception e) {
+        } catch (final ProcessException e) {
             logger.error("Unable to {} {} using {} compression format due to {}; routing to failure", new Object[]{compressionMode.toLowerCase(), flowFile, compressionFormat, e});
             session.transfer(flowFile, REL_FAILURE);
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0f8d00d5/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
index 827653b..9f8a16c 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java
@@ -28,23 +28,23 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.stream.io.NullOutputStream;
-import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.annotation.documentation.CapabilityDescription;
-import org.apache.nifi.annotation.behavior.EventDriven;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.stream.io.NullOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.ObjectHolder;
 
 @EventDriven
@@ -143,7 +143,7 @@ public class HashContent extends AbstractProcessor {
             logger.info("Successfully added attribute '{}' to {} with a value of {}; routing to success", new Object[]{attributeName, flowFile, hashValueHolder.get()});
             session.getProvenanceReporter().modifyAttributes(flowFile);
             session.transfer(flowFile, REL_SUCCESS);
-        } catch (final Exception e) {
+        } catch (final ProcessException e) {
             logger.error("Failed to process {} due to {}; routing to failure", new Object[]{flowFile, e});
             session.transfer(flowFile, REL_FAILURE);
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0f8d00d5/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
index 2fa71c8..eb6b1cc 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java
@@ -32,6 +32,7 @@ import java.util.Set;
 import javax.activation.DataHandler;
 import javax.mail.Message;
 import javax.mail.Message.RecipientType;
+import javax.mail.MessagingException;
 import javax.mail.Session;
 import javax.mail.URLName;
 import javax.mail.internet.AddressException;
@@ -56,9 +57,9 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
-
 import org.apache.commons.codec.binary.Base64;
 
 import com.sun.mail.smtp.SMTPTransport;
@@ -263,7 +264,7 @@ public class PutEmail extends AbstractProcessor {
             session.getProvenanceReporter().send(flowFile, "mailto:" + message.getAllRecipients()[0].toString());
             session.transfer(flowFile, REL_SUCCESS);
             logger.info("Sent email as a result of receiving {}", new Object[]{flowFile});
-        } catch (final Exception e) {
+        } catch (final ProcessException | MessagingException | IOException e) {
             context.yield();
             logger.error("Failed to send email for {}: {}; routing to failure", new Object[]{flowFile, e});
             session.transfer(flowFile, REL_FAILURE);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/0f8d00d5/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
index 5e251f6..8a2feb8 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java
@@ -51,6 +51,7 @@ import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.SideEffectFree;
 import org.apache.nifi.annotation.behavior.SupportsBatching;
 import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.StreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.util.StopWatch;
@@ -152,7 +153,7 @@ public class TransformXml extends AbstractProcessor {
             session.transfer(transformed, REL_SUCCESS);
             session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
             logger.info("Transformed {}", new Object[]{original});
-        } catch (Exception e) {
+        } catch (ProcessException e) {
             logger.error("Unable to transform {} due to {}", new Object[]{original, e});
             session.transfer(original, REL_FAILURE);
         }


[26/51] [abbrv] incubator-nifi git commit: NIFI-344 - Validating remote process group URI in the backend

Posted by mc...@apache.org.
NIFI-344 - Validating remote process group URI in the backend


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

Branch: refs/heads/NIFI-353
Commit: 09090825dea376abd5a4c962fa2034e952b7dca1
Parents: dde5fd5
Author: danbress <db...@onyxconsults.com>
Authored: Tue Feb 17 12:39:20 2015 -0500
Committer: danbress <db...@onyxconsults.com>
Committed: Tue Feb 17 12:39:20 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/web/api/RemoteProcessGroupResource.java  | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/09090825/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
index 6a5b536..3ddebd1 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java
@@ -370,6 +370,10 @@ public class RemoteProcessGroupResource extends ApplicationResource {
         if (uri.getScheme() == null || uri.getHost() == null) {
             throw new IllegalArgumentException("The specified remote process group URL is malformed: " + requestProcessGroupDTO.getTargetUri());
         }
+        
+        if (!(uri.getScheme().equalsIgnoreCase("http") || uri.getScheme().equalsIgnoreCase("https"))) {
+            throw new IllegalArgumentException("The specified remote process group URL is invalid, only http and https are supported: " + requestProcessGroupDTO.getTargetUri());
+        }
 
         // normalize the uri to the other controller
         String controllerUri = uri.toString();
@@ -860,6 +864,10 @@ public class RemoteProcessGroupResource extends ApplicationResource {
             if (uri.getScheme() == null || uri.getHost() == null) {
                 throw new IllegalArgumentException("The specified remote process group URL is malformed: " + requestRemoteProcessGroup.getTargetUri());
             }
+            
+            if (!(uri.getScheme().equalsIgnoreCase("http") || uri.getScheme().equalsIgnoreCase("https"))) {
+                throw new IllegalArgumentException("The specified remote process group URL is invalid, only http and https are supported: " + requestRemoteProcessGroup.getTargetUri());
+            }
 
             // normalize the uri to the other controller
             String controllerUri = uri.toString();


[47/51] [abbrv] incubator-nifi git commit: NIFI-383: Ensure that we always clean up sockets

Posted by mc...@apache.org.
NIFI-383: Ensure that we always clean up sockets


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

Branch: refs/heads/NIFI-353
Commit: ca23ad8eaa236407daff926d49c73985ed16a41e
Parents: 5aef55b
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 25 14:06:58 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 25 14:06:58 2015 -0500

----------------------------------------------------------------------
 .../client/socket/EndpointConnectionPool.java   |  5 ----
 .../SocketChannelCommunicationsSession.java     | 22 +++++++++++++-
 .../SSLSocketChannelCommunicationsSession.java  | 22 +++++++++++++-
 .../io/socket/SocketChannelInputStream.java     |  2 ++
 .../remote/io/socket/ssl/SSLSocketChannel.java  |  2 ++
 .../nifi/remote/StandardRemoteGroupPort.java    | 30 +++++++++++++-------
 6 files changed, 66 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ca23ad8e/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 42428f6..885f357 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
@@ -377,11 +377,6 @@ public class EndpointConnectionPool {
     		return false;
     	}
     	
-    	final String url = peer.getUrl();
-    	if ( url == null ) {
-    		return false;
-    	}
-    	
     	final BlockingQueue<EndpointConnection> connectionQueue = connectionQueueMap.get(peer.getDescription());
     	if ( connectionQueue == null ) {
     	    return false;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ca23ad8e/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
index 0822b6a..8065f57 100644
--- 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
@@ -64,7 +64,27 @@ public class SocketChannelCommunicationsSession extends AbstractCommunicationsSe
 
     @Override
     public void close() throws IOException {
-        channel.close();
+        IOException suppressed = null;
+        
+        try {
+            request.consume();
+        } catch (final IOException ioe) {
+            suppressed = ioe;
+        }
+        
+        try {
+            channel.close();
+        } catch (final IOException ioe) {
+            if ( suppressed != null ) {
+                ioe.addSuppressed(suppressed);
+            }
+            
+            throw ioe;
+        }
+        
+        if ( suppressed != null ) {
+            throw suppressed;
+        }
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ca23ad8e/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
index dca1d84..50e9162 100644
--- 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
@@ -54,7 +54,27 @@ public class SSLSocketChannelCommunicationsSession extends AbstractCommunication
 
     @Override
     public void close() throws IOException {
-        channel.close();
+        IOException suppressed = null;
+        
+        try {
+            request.consume();
+        } catch (final IOException ioe) {
+            suppressed = ioe;
+        }
+        
+        try {
+            channel.close();
+        } catch (final IOException ioe) {
+            if ( suppressed != null ) {
+                ioe.addSuppressed(suppressed);
+            }
+            
+            throw ioe;
+        }
+        
+        if ( suppressed != null ) {
+            throw suppressed;
+        }
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ca23ad8e/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 f68c874..0ad0b74 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
@@ -45,6 +45,8 @@ public class SocketChannelInputStream extends InputStream {
     }
     
     public void consume() throws IOException {
+        channel.shutdownInput();
+        
         final byte[] b = new byte[4096];
         final ByteBuffer buffer = ByteBuffer.wrap(b);
         int bytesRead;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ca23ad8e/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 7c74b20..249ad48 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
@@ -259,6 +259,8 @@ public class SSLSocketChannel implements Closeable {
     }
 
     public void consume() throws IOException {
+        channel.shutdownInput();
+        
         final byte[] b = new byte[4096];
         final ByteBuffer buffer = ByteBuffer.wrap(b);
         int readCount;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ca23ad8e/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 da9d027..740e405 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
@@ -150,6 +150,18 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         
         String url = getRemoteProcessGroup().getTargetUri().toString();
         
+        // If we are sending data, we need to ensure that we have at least 1 FlowFile to send. Otherwise,
+        // we don't want to create a transaction at all.
+        final FlowFile firstFlowFile;
+        if ( getConnectableType() == ConnectableType.REMOTE_INPUT_PORT ) {
+            firstFlowFile = session.get();
+            if ( firstFlowFile == null ) {
+                return;
+            }
+        } else {
+            firstFlowFile = null;
+        }
+        
         final SiteToSiteClient client = clientRef.get();
         final Transaction transaction;
         try {
@@ -187,7 +199,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
 
         try {
             if ( getConnectableType() == ConnectableType.REMOTE_INPUT_PORT ) {
-                transferFlowFiles(transaction, context, session);
+                transferFlowFiles(transaction, context, session, firstFlowFile);
             } else {
                 final int numReceived = receiveFlowFiles(transaction, context, session);
                 if ( numReceived == 0 ) {
@@ -196,14 +208,15 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
             }
 
             session.commit();
-        } catch (final Exception e) {
-            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());
+        } catch (final Throwable t) {
+            final String message = String.format("%s failed to communicate with remote NiFi instance due to %s", this, t.toString());
+            logger.error("{} failed to communicate with remote NiFi instance due to {}", this, t.toString());
             if ( logger.isDebugEnabled() ) {
-                logger.error("", e);
+                logger.error("", t);
             }
             
             remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message);
+            transaction.error();
             session.rollback();
         }
     }
@@ -216,11 +229,8 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
     }
     
     
-    private int transferFlowFiles(final Transaction transaction, final ProcessContext context, final ProcessSession session) throws IOException, ProtocolException {
-        FlowFile flowFile = session.get();
-        if (flowFile == null) {
-            return 0;
-        }
+    private int transferFlowFiles(final Transaction transaction, final ProcessContext context, final ProcessSession session, FlowFile firstFlowFile) throws IOException, ProtocolException {
+        FlowFile flowFile = firstFlowFile;
 
         try {
             final String userDn = transaction.getCommunicant().getDistinguishedName();


[24/51] [abbrv] incubator-nifi git commit: NIFI-282: Allow Builder to be populated from SiteToSiteClientConfig object

Posted by mc...@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/NIFI-353
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");
         


[37/51] [abbrv] 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 mc...@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/NIFI-353
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


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

Posted by mc...@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/NIFI-353
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(-)
----------------------------------------------------------------------



[34/51] [abbrv] incubator-nifi git commit: Merge branch 'develop' into nifi-site-to-site-client

Posted by mc...@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/NIFI-353
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
----------------------------------------------------------------------


[49/51] [abbrv] incubator-nifi git commit: NIFI-386: removed inadvertently added bin/ files

Posted by mc...@apache.org.
NIFI-386: removed inadvertently added bin/ files


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

Branch: refs/heads/NIFI-353
Commit: 97aa53da66048958ebc89b442bc7d2b59e596c73
Parents: 1af8c1e
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Feb 25 14:23:21 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Feb 25 14:23:21 2015 -0500

----------------------------------------------------------------------
 .../controller/util/RemoteProcessGroupUtils.class  | Bin 9526 -> 0 bytes
 .../remote/AbstractCommunicationsSession.class     | Bin 2308 -> 0 bytes
 .../apache/nifi/remote/RemoteResourceFactory.class | Bin 8707 -> 0 bytes
 .../apache/nifi/remote/RemoteResourceManager.class | Bin 6898 -> 0 bytes
 .../apache/nifi/remote/RemoteSiteListener.class    | Bin 841 -> 0 bytes
 .../nifi/remote/SocketRemoteSiteListener.class     | Bin 8448 -> 0 bytes
 ...rdRemoteGroupPort$EndpointConnectionState.class | Bin 5427 -> 0 bytes
 .../nifi/remote/StandardRemoteGroupPort.class      | Bin 10677 -> 0 bytes
 .../StandardRootGroupPort$FlowFileRequest.class    | Bin 5836 -> 0 bytes
 .../StandardRootGroupPort$ProcessingResult.class   | Bin 5032 -> 0 bytes
 ...GroupPort$StandardPortAuthorizationResult.class | Bin 5159 -> 0 bytes
 .../apache/nifi/remote/StandardRootGroupPort.class | Bin 9700 -> 0 bytes
 .../nifi/remote/codec/StandardFlowFileCodec.class  | Bin 8538 -> 0 bytes
 .../exception/UnsupportedCodecException.class      | Bin 1057 -> 0 bytes
 .../SocketChannelCommunicationsSession.class       | Bin 3735 -> 0 bytes
 .../nifi/remote/io/socket/SocketChannelInput.class | Bin 4008 -> 0 bytes
 .../remote/io/socket/SocketChannelOutput.class     | Bin 3741 -> 0 bytes
 .../SSLSocketChannelCommunicationsSession.class    | Bin 4611 -> 0 bytes
 .../io/socket/ssl/SSLSocketChannelInput.class      | Bin 3127 -> 0 bytes
 .../io/socket/ssl/SSLSocketChannelOutput.class     | Bin 2587 -> 0 bytes
 .../socket/ClusterManagerServerProtocol.class      | Bin 10540 -> 0 bytes
 .../remote/protocol/socket/HandshakeProperty.class | Bin 917 -> 0 bytes
 .../nifi/remote/protocol/socket/Response.class     | Bin 2674 -> 0 bytes
 .../nifi/remote/protocol/socket/ResponseCode.class | Bin 6889 -> 0 bytes
 .../protocol/socket/SocketClientProtocol.class     | Bin 8965 -> 0 bytes
 .../socket/SocketFlowFileServerProtocol.class      | Bin 8806 -> 0 bytes
 .../nifi/remote/TestStandardRemoteGroupPort.class  | Bin 5974 -> 0 bytes
 27 files changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index a6951d4..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 2e868ea..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 9c6e821..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index bb3fc77..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index a56b5ba..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 9780f75..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index f184d64..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 0740bb4..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index d517458..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 1cf5ceb..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 3ad7542..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 4db4735..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index e49ffe8..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 39dd49a..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index b415421..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 551097e..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 6913767..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index f6e9f20..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 41fe366..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 9b9cdc0..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 2ef1c39..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index fad8245..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 27ec8d3..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 4673aec..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index ac7e1b9..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 933ef4d..0000000
Binary files 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 and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/97aa53da/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
deleted file mode 100644
index 0e77276..0000000
Binary files a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class and /dev/null differ