You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2017/03/06 16:18:12 UTC

[1/2] nifi git commit: NIFI-3541: Add local network interface capability to site-to-site client and remote group and ports

Repository: nifi
Updated Branches:
  refs/heads/master 000414e7e -> 16bde02ed


NIFI-3541: Add local network interface capability to site-to-site client and remote group and ports


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

Branch: refs/heads/master
Commit: 9e68f02f1fb7eb442f6c9580b46255f713d8b191
Parents: 000414e
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Mar 1 13:30:04 2017 -0500
Committer: Aldrin Piri <al...@apache.org>
Committed: Mon Mar 6 10:36:30 2017 -0500

----------------------------------------------------------------------
 .../nifi/remote/client/SiteToSiteClient.java    |  40 +++++-
 .../remote/client/SiteToSiteClientConfig.java   |   7 ++
 .../remote/util/SiteToSiteRestApiClient.java    | 109 ++++++++++-------
 .../apache/nifi/controller/AbstractPort.java    |  32 ++---
 .../apache/nifi/groups/RemoteProcessGroup.java  |  24 ++++
 .../org/apache/nifi/remote/RemoteGroupPort.java |   1 +
 .../nifi/remote/StandardRemoteProcessGroup.java | 122 ++++++++++++++-----
 .../nifi/remote/StandardRemoteGroupPort.java    |  27 +++-
 .../remote/TestStandardRemoteGroupPort.java     |   1 +
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  66 ++++++----
 10 files changed, 303 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
index 3d7bacc..daff70d 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -39,6 +39,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.Serializable;
+import java.net.InetAddress;
 import java.security.KeyStore;
 import java.security.SecureRandom;
 import java.util.LinkedHashSet;
@@ -168,6 +169,7 @@ public interface SiteToSiteClient extends Closeable {
         private int batchCount;
         private long batchSize;
         private long batchNanos;
+        private InetAddress localAddress;
         private SiteToSiteTransportProtocol transportProtocol = SiteToSiteTransportProtocol.RAW;
         private HttpProxy httpProxy;
 
@@ -198,6 +200,7 @@ public interface SiteToSiteClient extends Closeable {
             this.batchCount = config.getPreferredBatchCount();
             this.batchSize = config.getPreferredBatchSize();
             this.batchNanos = config.getPreferredBatchDuration(TimeUnit.NANOSECONDS);
+            this.localAddress = config.getLocalAddress();
             this.httpProxy = config.getHttpProxy();
 
             return this;
@@ -223,12 +226,31 @@ public interface SiteToSiteClient extends Closeable {
         }
 
         /**
-         * <p>Specifies the URLs of the remote NiFi instance.</p>
-         * <p>If this URL points to a NiFi node in a NiFi cluster, data transfer to and from
-         * nodes will be automatically load balanced across the different nodes.</p>
+         * <p>
+         * Specifies the local address to use when communicating with the remote NiFi instance.
+         * </p>
+         *
+         * @param localAddress the local address to use, or <code>null</code> to use <code>anyLocal</code> address.
+         * @return the builder
+         */
+        public Builder localAddress(final InetAddress localAddress) {
+            this.localAddress = localAddress;
+            return this;
+        }
+
+        /**
+         * <p>
+         * Specifies the URLs of the remote NiFi instance.
+         * </p>
+         * <p>
+         * If this URL points to a NiFi node in a NiFi cluster, data transfer to and from
+         * nodes will be automatically load balanced across the different nodes.
+         * </p>
          *
-         * <p>Multiple urls provide better connectivity with a NiFi cluster, able to connect
-         * to the target cluster at long as one of the specified urls is accessible.</p>
+         * <p>
+         * Multiple urls provide better connectivity with a NiFi cluster, able to connect
+         * to the target cluster at long as one of the specified urls is accessible.
+         * </p>
          *
          * @param urls urls of remote instance
          * @return the builder
@@ -717,6 +739,7 @@ public interface SiteToSiteClient extends Closeable {
         private final long batchSize;
         private final long batchNanos;
         private final HttpProxy httpProxy;
+        private final InetAddress localAddress;
 
         // some serialization frameworks require a default constructor
         private StandardSiteToSiteClientConfig() {
@@ -740,6 +763,7 @@ public interface SiteToSiteClient extends Closeable {
             this.batchNanos = 0;
             this.transportProtocol = null;
             this.httpProxy = null;
+            this.localAddress = null;
         }
 
         private StandardSiteToSiteClientConfig(final SiteToSiteClient.Builder builder) {
@@ -766,6 +790,7 @@ public interface SiteToSiteClient extends Closeable {
             this.batchNanos = builder.batchNanos;
             this.transportProtocol = builder.getTransportProtocol();
             this.httpProxy = builder.getHttpProxy();
+            this.localAddress = builder.localAddress;
         }
 
         @Override
@@ -931,5 +956,10 @@ public interface SiteToSiteClient extends Closeable {
         public HttpProxy getHttpProxy() {
             return httpProxy;
         }
+
+        @Override
+        public InetAddress getLocalAddress() {
+            return localAddress;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
index 5bdeee4..83e8328 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
@@ -18,6 +18,7 @@ package org.apache.nifi.remote.client;
 
 import java.io.File;
 import java.io.Serializable;
+import java.net.InetAddress;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
@@ -36,6 +37,7 @@ public interface SiteToSiteClientConfig extends Serializable {
      * for backward compatibility for implementations that does not expect multiple URLs.
      * {@link #getUrls()} should be used instead then should support multiple URLs when making requests.
      */
+    @Deprecated
     String getUrl();
 
     /**
@@ -171,4 +173,9 @@ public interface SiteToSiteClientConfig extends Serializable {
      */
     HttpProxy getHttpProxy();
 
+    /**
+     * @return the InetAddress to bind to for the local address when creating a socket, or
+     *         {@code null} to bind to the {@code anyLocal} address.
+     */
+    InetAddress getLocalAddress();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java
index 89da6a0..e6777b0 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/util/SiteToSiteRestApiClient.java
@@ -16,6 +16,55 @@
  */
 package org.apache.nifi.remote.util;
 
+import static org.apache.commons.lang3.StringUtils.isEmpty;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_COUNT;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_DURATION;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_SIZE;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_REQUEST_EXPIRATION;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_HEADER_NAME;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
+import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.net.InetAddress;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.http.Header;
 import org.apache.http.HttpEntity;
@@ -87,53 +136,6 @@ import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLPeerUnverifiedException;
-import javax.net.ssl.SSLSession;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.PipedInputStream;
-import java.io.PipedOutputStream;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.security.cert.Certificate;
-import java.security.cert.CertificateException;
-import java.security.cert.X509Certificate;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashSet;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Predicate;
-import java.util.regex.Pattern;
-
-import static org.apache.commons.lang3.StringUtils.isEmpty;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_COUNT;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_DURATION;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_BATCH_SIZE;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_REQUEST_EXPIRATION;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.HANDSHAKE_PROPERTY_USE_COMPRESSION;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_HEADER_NAME;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_NAME;
-import static org.apache.nifi.remote.protocol.http.HttpHeaders.LOCATION_URI_INTENT_VALUE;
-
 public class SiteToSiteRestApiClient implements Closeable {
 
     private static final String EVENT_CATEGORY = "Site-to-Site";
@@ -160,6 +162,7 @@ public class SiteToSiteRestApiClient implements Closeable {
     private CloseableHttpAsyncClient httpAsyncClient;
 
     private boolean compress = false;
+    private InetAddress localAddress = null;
     private long requestExpirationMillis = 0;
     private int serverTransactionTtl = 0;
     private int batchCount = 0;
@@ -239,6 +242,10 @@ public class SiteToSiteRestApiClient implements Closeable {
             .setConnectTimeout(connectTimeoutMillis)
             .setSocketTimeout(readTimeoutMillis);
 
+        if (localAddress != null) {
+            requestConfigBuilder.setLocalAddress(localAddress);
+        }
+        
         if (proxy != null) {
             requestConfigBuilder.setProxy(proxy.getHttpHost());
         }
@@ -916,6 +923,8 @@ public class SiteToSiteRestApiClient implements Closeable {
         extendingApiClient.transportProtocolVersionNegotiator = this.transportProtocolVersionNegotiator;
         extendingApiClient.connectTimeoutMillis = this.connectTimeoutMillis;
         extendingApiClient.readTimeoutMillis = this.readTimeoutMillis;
+        extendingApiClient.localAddress = this.localAddress;
+
         final int extendFrequency = serverTransactionTtl / 2;
 
         ttlExtendingFuture = ttlExtendTaskExecutor.scheduleWithFixedDelay(() -> {
@@ -1197,10 +1206,12 @@ public class SiteToSiteRestApiClient implements Closeable {
 
     public void setConnectTimeoutMillis(final int connectTimeoutMillis) {
         this.connectTimeoutMillis = connectTimeoutMillis;
+        setupRequestConfig();
     }
 
     public void setReadTimeoutMillis(final int readTimeoutMillis) {
         this.readTimeoutMillis = readTimeoutMillis;
+        setupRequestConfig();
     }
 
     public static String getFirstUrl(final String clusterUrlStr) {
@@ -1336,6 +1347,10 @@ public class SiteToSiteRestApiClient implements Closeable {
     public void setCompress(final boolean compress) {
         this.compress = compress;
     }
+    
+    public void setLocalAddress(final InetAddress localAddress) {
+        this.localAddress = localAddress;
+    }
 
     public void setRequestExpirationMillis(final long requestExpirationMillis) {
         if (requestExpirationMillis < 0) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
index 1177dad..4d061b8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java
@@ -16,6 +16,22 @@
  */
 package org.apache.nifi.controller;
 
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
 import org.apache.nifi.authorization.Resource;
@@ -36,22 +52,6 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.FormatUtils;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static java.util.Objects.requireNonNull;
-
 public abstract class AbstractPort implements Port {
 
     public static final Relationship PORT_RELATIONSHIP = new Relationship.Builder()

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
index 64e2ca0..cb1e6c8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java
@@ -17,18 +17,22 @@
 package org.apache.nifi.groups;
 
 import org.apache.nifi.authorization.resource.ComponentAuthorizable;
+import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.connectable.Positionable;
 import org.apache.nifi.controller.exception.CommunicationsException;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 
+import java.net.InetAddress;
+import java.util.Collection;
 import java.util.Date;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 public interface RemoteProcessGroup extends ComponentAuthorizable, Positionable {
 
+    @Override
     String getIdentifier();
 
     String getTargetUri();
@@ -155,6 +159,16 @@ public interface RemoteProcessGroup extends ComponentAuthorizable, Positionable
     String getAuthorizationIssue();
 
     /**
+     * Validates the current configuration, returning ValidationResults for any
+     * invalid configuration parameter.
+     *
+     * @return Collection of validation result objects for any invalid findings
+     *         only. If the collection is empty then the component is valid. Guaranteed
+     *         non-null
+     */
+    Collection<ValidationResult> validate();
+
+    /**
      * @return the {@link EventReporter} that can be used to report any notable
      * events
      */
@@ -180,6 +194,16 @@ public interface RemoteProcessGroup extends ComponentAuthorizable, Positionable
 
     void setProxyPassword(String proxyPassword);
 
+    void setNetworkInterface(String interfaceName);
+
+    String getNetworkInterface();
+
+    /**
+     * Returns the InetAddress that the will this instance will bind to when communicating with a
+     * remote NiFi instance, or <code>null</code> if no specific address has been specified
+     */
+    InetAddress getLocalAddress();
+
     /**
      * Initiates a task in the remote process group to re-initialize, as a
      * result of clustering changes

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
index 8cad103..f8f4b20 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/RemoteGroupPort.java
@@ -33,6 +33,7 @@ public abstract class RemoteGroupPort extends AbstractPort implements Port, Remo
 
     public abstract TransferDirection getTransferDirection();
 
+    @Override
     public abstract boolean isUseCompression();
 
     public abstract void setUseCompression(boolean useCompression);

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 855dab7..67c8f11 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -18,14 +18,15 @@ package org.apache.nifi.remote;
 
 import static java.util.Objects.requireNonNull;
 
-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.UniformInterfaceException;
 import java.io.File;
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
+import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -40,12 +41,15 @@ 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 javax.net.ssl.SSLContext;
 import javax.ws.rs.core.Response;
+
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.resource.ResourceType;
+import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Port;
@@ -74,6 +78,11 @@ import org.apache.nifi.web.api.dto.PortDTO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+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.UniformInterfaceException;
+
 /**
  * Represents the Root Process Group of a remote NiFi Instance. Holds
  * information about that remote instance, as well as {@link IncomingPort}s and
@@ -99,7 +108,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     private final AtomicReference<String> comments = new AtomicReference<>();
     private final AtomicReference<ProcessGroup> processGroup;
     private final AtomicBoolean transmitting = new AtomicBoolean(false);
-    private final FlowController flowController;
     private final SSLContext sslContext;
 
     private volatile String communicationsTimeout = "30 sec";
@@ -111,6 +119,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     private volatile String proxyUser;
     private volatile String proxyPassword;
 
+    private String networkInterfaceName;
+    private InetAddress localAddress;
+    private ValidationResult nicValidationResult;
+
+
     private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
     private final Lock readLock = rwLock.readLock();
     private final Lock writeLock = rwLock.writeLock();
@@ -135,7 +148,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                                       final FlowController flowController, final SSLContext sslContext, final NiFiProperties nifiProperties) {
         this.nifiProperties = nifiProperties;
         this.id = requireNonNull(id);
-        this.flowController = requireNonNull(flowController);
 
         this.targetUris = targetUris;
         this.targetId = null;
@@ -354,6 +366,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         return authorizationIssue;
     }
 
+    @Override
+    public Collection<ValidationResult> validate() {
+        return (nicValidationResult == null) ? Collections.emptyList() : Collections.singletonList(nicValidationResult);
+    }
+
     public int getInputPortCount() {
         readLock.lock();
         try {
@@ -606,7 +623,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
             }
 
             final StandardRemoteGroupPort port = new StandardRemoteGroupPort(descriptor.getId(), descriptor.getName(), getProcessGroup(),
-                    this, TransferDirection.RECEIVE, ConnectableType.REMOTE_OUTPUT_PORT, sslContext, scheduler, nifiProperties);
+                this, TransferDirection.RECEIVE, ConnectableType.REMOTE_OUTPUT_PORT, sslContext, scheduler, nifiProperties);
             outputPorts.put(descriptor.getId(), port);
 
             if (descriptor.getConcurrentlySchedulableTaskCount() != null) {
@@ -672,7 +689,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
             }
 
             final StandardRemoteGroupPort port = new StandardRemoteGroupPort(descriptor.getId(), descriptor.getName(), getProcessGroup(), this,
-                    TransferDirection.SEND, ConnectableType.REMOTE_INPUT_PORT, sslContext, scheduler, nifiProperties);
+                TransferDirection.SEND, ConnectableType.REMOTE_INPUT_PORT, sslContext, scheduler, nifiProperties);
 
             if (descriptor.getConcurrentlySchedulableTaskCount() != null) {
                 port.setMaxConcurrentTasks(descriptor.getConcurrentlySchedulableTaskCount());
@@ -741,15 +758,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         }
     }
 
-    private ProcessGroup getRootGroup() {
-        return getRootGroup(getProcessGroup());
-    }
-
-    private ProcessGroup getRootGroup(final ProcessGroup context) {
-        final ProcessGroup parent = context.getParent();
-        return parent == null ? context : getRootGroup(parent);
-    }
-
     @Override
     public Date getLastRefreshTime() {
         readLock.lock();
@@ -856,10 +864,75 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         }
     }
 
+    @Override
+    public String getNetworkInterface() {
+        readLock.lock();
+        try {
+            return networkInterfaceName;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    @Override
+    public void setNetworkInterface(final String interfaceName) {
+        writeLock.lock();
+        try {
+            this.networkInterfaceName = interfaceName;
+
+            try {
+                final Enumeration<InetAddress> inetAddresses = NetworkInterface.getByName(interfaceName).getInetAddresses();
+
+                if (inetAddresses.hasMoreElements()) {
+                    this.localAddress = inetAddresses.nextElement();
+                    this.nicValidationResult = null;
+                } else {
+                    this.localAddress = null;
+                    this.nicValidationResult = new ValidationResult.Builder()
+                        .input(interfaceName)
+                        .subject("Network Interface Name")
+                        .valid(false)
+                        .explanation("No IP Address could be found that is bound to the interface with name " + interfaceName)
+                        .build();
+                }
+            } catch (final Exception e) {
+                this.localAddress = null;
+                this.nicValidationResult = new ValidationResult.Builder()
+                    .input(interfaceName)
+                    .subject("Network Interface Name")
+                    .valid(false)
+                    .explanation("Could not obtain Network Interface with name " + interfaceName)
+                    .build();
+            }
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public InetAddress getLocalAddress() {
+        readLock.lock();
+        try {
+            if (nicValidationResult != null && !nicValidationResult.isValid()) {
+                return null;
+            }
+
+            return localAddress;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
     private SiteToSiteRestApiClient getSiteToSiteRestApiClient() {
         SiteToSiteRestApiClient apiClient = new SiteToSiteRestApiClient(sslContext, new HttpProxy(proxyHost, proxyPort, proxyUser, proxyPassword), getEventReporter());
         apiClient.setConnectTimeoutMillis(getCommunicationsTimeout(TimeUnit.MILLISECONDS));
         apiClient.setReadTimeoutMillis(getCommunicationsTimeout(TimeUnit.MILLISECONDS));
+
+        final InetAddress localAddress = getLocalAddress();
+        if (localAddress != null) {
+            apiClient.setLocalAddress(localAddress);
+        }
+
         return apiClient;
     }
 
@@ -886,17 +959,6 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         return remotePorts;
     }
 
-    private RemoteProcessGroupPortDescriptor convertPortToRemotePortDescriptor(final Port port) {
-        final StandardRemoteProcessGroupPortDescriptor descriptor = new StandardRemoteProcessGroupPortDescriptor();
-        descriptor.setComments(port.getComments());
-        descriptor.setExists(true);
-        descriptor.setGroupId(port.getProcessGroup().getIdentifier());
-        descriptor.setId(port.getIdentifier());
-        descriptor.setName(port.getName());
-        descriptor.setTargetRunning(port.isRunning());
-        return descriptor;
-    }
-
     @Override
     public boolean isTransmitting() {
         return transmitting.get();
@@ -1216,6 +1278,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                 if (port.hasIncomingConnection() && !port.getTargetExists()) {
                     throw new IllegalStateException(this.getIdentifier() + " has a Connection to Port " + port.getIdentifier() + ", but that Port no longer exists on the remote system");
                 }
+
+                port.verifyCanStart();
             }
 
             for (final StandardRemoteGroupPort port : outputPorts.values()) {
@@ -1226,6 +1290,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
                 if (!port.getConnections().isEmpty() && !port.getTargetExists()) {
                     throw new IllegalStateException(this.getIdentifier() + " has a Connection to Port " + port.getIdentifier() + ", but that Port no longer exists on the remote system");
                 }
+
+                port.verifyCanStart();
             }
         } finally {
             readLock.unlock();

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
index d8c055b..92931f2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java
@@ -24,6 +24,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -93,7 +94,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
 
     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,
-            final NiFiProperties nifiProperties) {
+        final NiFiProperties nifiProperties) {
         // remote group port id needs to be unique but cannot just be the id of the port
         // in the remote group instance. this supports referencing the same remote
         // instance more than once.
@@ -167,6 +168,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
                 .timeout(remoteGroup.getCommunicationsTimeout(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
                 .transportProtocol(remoteGroup.getTransportProtocol())
                 .httpProxy(new HttpProxy(remoteGroup.getProxyHost(), remoteGroup.getProxyPort(), remoteGroup.getProxyUser(), remoteGroup.getProxyPassword()))
+                .localAddress(remoteGroup.getLocalAddress())
                 .build();
         clientRef.set(client);
     }
@@ -407,8 +409,19 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
 
     @Override
     public boolean isValid() {
-        return targetExists.get()
-                && (getConnectableType() == ConnectableType.REMOTE_OUTPUT_PORT ? !getConnections(Relationship.ANONYMOUS).isEmpty() : true);
+        if (!targetExists.get()) {
+            return false;
+        }
+
+        if (getConnectableType() == ConnectableType.REMOTE_OUTPUT_PORT && getConnections(Relationship.ANONYMOUS).isEmpty()) {
+            // if it's an output port, ensure that there is an outbound connection
+            return false;
+        }
+
+        final boolean groupValid = remoteGroup.validate().stream()
+            .allMatch(result -> result.isValid());
+
+        return groupValid;
     }
 
     @Override
@@ -444,6 +457,14 @@ public class StandardRemoteGroupPort extends RemoteGroupPort {
         if (getConnectableType() == ConnectableType.REMOTE_INPUT_PORT && getIncomingConnections().isEmpty()) {
             throw new IllegalStateException("Port " + getName() + " has no incoming connections");
         }
+
+        final Optional<ValidationResult> resultOption = remoteGroup.validate().stream()
+            .filter(result -> !result.isValid())
+            .findFirst();
+
+        if (resultOption.isPresent()) {
+            throw new IllegalStateException("Remote Process Group is not valid: " + resultOption.get().toString());
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
index 2d48515..31cd154 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
@@ -110,6 +110,7 @@ public class TestStandardRemoteGroupPort {
                 connectableType = null;
                 break;
         }
+
         port = spy(new StandardRemoteGroupPort(ID, NAME,
                 processGroup, remoteGroup, direction, connectableType, null, scheduler, NiFiProperties.createBasicNiFiProperties(null, null)));
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/9e68f02f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 8049c12..113d491 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -16,6 +16,32 @@
  */
 package org.apache.nifi.web.api.dto;
 
+import java.text.Collator;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.WebApplicationException;
+
 import org.apache.nifi.action.Action;
 import org.apache.nifi.action.component.details.ComponentDetails;
 import org.apache.nifi.action.component.details.ExtensionDetails;
@@ -157,32 +183,6 @@ import org.apache.nifi.web.api.entity.TenantEntity;
 import org.apache.nifi.web.controller.ControllerFacade;
 import org.apache.nifi.web.revision.RevisionManager;
 
-import javax.ws.rs.WebApplicationException;
-import java.text.Collator;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
 public final class DtoFactory {
 
     @SuppressWarnings("rawtypes")
@@ -1550,7 +1550,19 @@ public final class DtoFactory {
         }
 
         if (group.getAuthorizationIssue() != null) {
-            dto.setAuthorizationIssues(Arrays.asList(group.getAuthorizationIssue()));
+            final List<String> authIssues = new ArrayList<>();
+            final String authIssue = group.getAuthorizationIssue();
+            if (authIssue != null) {
+                authIssues.add(authIssue);
+            }
+
+            final Collection<ValidationResult> validationResults = group.validate();
+            validationResults.stream()
+                .filter(result -> !result.isValid())
+                .map(result -> result.toString())
+                .forEach(str -> authIssues.add(str));
+
+            dto.setAuthorizationIssues(authIssues);
         }
 
         dto.setActiveRemoteInputPortCount(activeRemoteInputPortCount);


[2/2] nifi git commit: NIFI-3541: - Allowing the user to specify the network interface to send/receive data for a Remote Process Group.

Posted by al...@apache.org.
NIFI-3541: - Allowing the user to specify the network interface to send/receive data for a Remote Process Group.

This closes #1550.

Signed-off-by: Mark Payne <ma...@hotmail.com>
Signed-off-by: Aldrin Piri <al...@apache.org>


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

Branch: refs/heads/master
Commit: 16bde02ed079150c1fbf80806537137613ae2d10
Parents: 9e68f02
Author: Matt Gilman <ma...@gmail.com>
Authored: Wed Mar 1 13:51:34 2017 -0500
Committer: Aldrin Piri <al...@apache.org>
Committed: Mon Mar 6 10:38:15 2017 -0500

----------------------------------------------------------------------
 .../remote/io/socket/ssl/SSLSocketChannel.java  |   7 +++-
 .../client/socket/EndpointConnectionPool.java   |  14 ++++++--
 .../nifi/remote/client/socket/SocketClient.java |   2 +-
 .../images/configure-remote-process-group.png   | Bin 49004 -> 52661 bytes
 nifi-docs/src/main/asciidoc/user-guide.adoc     |  14 +++++---
 .../nifi/web/api/dto/RemoteProcessGroupDTO.java |  21 +++++++++++
 .../manager/RemoteProcessGroupEntityMerger.java |   5 ++-
 .../nifi/remote/protocol/ServerProtocol.java    |   4 +--
 .../controller/StandardFlowSynchronizer.java    |   7 ++++
 .../serialization/FlowFromDOMFactory.java       |   1 +
 .../serialization/StandardFlowSerializer.java   |   5 ++-
 .../nifi/fingerprint/FingerprintFactory.java    |   1 +
 .../nifi/remote/StandardRemoteProcessGroup.java |  35 ++++++++++---------
 .../src/main/resources/FlowConfiguration.xsd    |   1 +
 .../StandardHttpFlowFileServerProtocol.java     |   2 +-
 .../socket/SocketFlowFileServerProtocol.java    |   4 +--
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  23 ++++++------
 .../dao/impl/StandardRemoteProcessGroupDAO.java |  24 +++++++++----
 .../canvas/new-remote-process-group-dialog.jsp  |  34 ++++++++++++------
 .../remote-process-group-configuration.jsp      |  34 ++++++++++++------
 .../canvas/remote-process-group-details.jsp     |  34 ++++++++++++------
 .../nifi-web-ui/src/main/webapp/css/dialog.css  |   4 ---
 .../css/remote-process-group-configuration.css  |  13 ++-----
 .../nf-ng-remote-process-group-component.js     |   5 +++
 .../nf-remote-process-group-configuration.js    |   5 ++-
 .../canvas/nf-remote-process-group-details.js   |   2 ++
 .../js/nf/canvas/nf-remote-process-group.js     |  29 +++++++++++----
 .../cache/client/SSLCommsSession.java           |   2 +-
 28 files changed, 226 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
index 7a09f5f..7e5b303 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java
@@ -18,6 +18,7 @@ package org.apache.nifi.remote.io.socket.ssl;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.SocketAddress;
@@ -68,9 +69,13 @@ public class SSLSocketChannel implements Closeable {
     private boolean closed = false;
     private volatile boolean interrupted = false;
 
-    public SSLSocketChannel(final SSLContext sslContext, final String hostname, final int port, final boolean client) throws IOException {
+    public SSLSocketChannel(final SSLContext sslContext, final String hostname, final int port, final InetAddress localAddress, final boolean client) throws IOException {
         this.socketAddress = new InetSocketAddress(hostname, port);
         this.channel = SocketChannel.open();
+        if (localAddress != null) {
+            final SocketAddress localSocketAddress = new InetSocketAddress(localAddress, 0);
+            this.channel.bind(localSocketAddress);
+        }
         this.hostname = hostname;
         this.port = port;
         this.engine = sslContext.createSSLEngine();

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
index 6f08f73..926e4b4 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java
@@ -23,7 +23,9 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.net.URI;
 import java.nio.channels.SocketChannel;
 import java.security.cert.CertificateException;
@@ -87,9 +89,11 @@ public class EndpointConnectionPool implements PeerStatusProvider {
 
     private final SiteInfoProvider siteInfoProvider;
     private final PeerSelector peerSelector;
+    private final InetAddress localAddress;
 
     public EndpointConnectionPool(final RemoteDestination remoteDestination, final int commsTimeoutMillis, final int idleExpirationMillis,
-            final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile, final SiteInfoProvider siteInfoProvider) {
+        final SSLContext sslContext, final EventReporter eventReporter, final File persistenceFile, final SiteInfoProvider siteInfoProvider,
+        final InetAddress localAddress) {
         Objects.requireNonNull(remoteDestination, "Remote Destination/Port Identifier cannot be null");
 
         this.remoteDestination = remoteDestination;
@@ -97,6 +101,7 @@ public class EndpointConnectionPool implements PeerStatusProvider {
         this.eventReporter = eventReporter;
         this.commsTimeout = commsTimeoutMillis;
         this.idleExpirationMillis = idleExpirationMillis;
+        this.localAddress = localAddress;
 
         this.siteInfoProvider = siteInfoProvider;
 
@@ -440,7 +445,7 @@ public class EndpointConnectionPool implements PeerStatusProvider {
                             + " 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);
+                final SSLSocketChannel socketChannel = new SSLSocketChannel(sslContext, hostname, port, localAddress, true);
                 socketChannel.connect();
 
                 commsSession = new SSLSocketChannelCommunicationsSession(socketChannel);
@@ -452,6 +457,11 @@ public class EndpointConnectionPool implements PeerStatusProvider {
                 }
             } else {
                 final SocketChannel socketChannel = SocketChannel.open();
+                if (localAddress != null) {
+                    final SocketAddress localSocketAddress = new InetSocketAddress(localAddress, 0);
+                    socketChannel.socket().bind(localSocketAddress);
+                }
+
                 socketChannel.socket().connect(new InetSocketAddress(hostname, port), commsTimeout);
                 socketChannel.socket().setSoTimeout(commsTimeout);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
index 1d3cce7..ba6839c 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java
@@ -52,7 +52,7 @@ public class SocketClient extends AbstractSiteToSiteClient {
                 commsTimeout,
                 (int) config.getIdleConnectionExpiration(TimeUnit.MILLISECONDS),
                 config.getSslContext(), config.getEventReporter(), config.getPeerPersistenceFile(),
-                siteInfoProvider
+                siteInfoProvider, config.getLocalAddress()
         );
 
         this.compress = config.isUseCompression();

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-docs/src/main/asciidoc/images/configure-remote-process-group.png
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/images/configure-remote-process-group.png b/nifi-docs/src/main/asciidoc/images/configure-remote-process-group.png
old mode 100755
new mode 100644
index 9a13c27..a1477da
Binary files a/nifi-docs/src/main/asciidoc/images/configure-remote-process-group.png and b/nifi-docs/src/main/asciidoc/images/configure-remote-process-group.png differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-docs/src/main/asciidoc/user-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc
index 2fe4374..085785e 100644
--- a/nifi-docs/src/main/asciidoc/user-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/user-guide.adoc
@@ -367,9 +367,9 @@ image:iconRemoteProcessGroup.png["Remote Process Group", width=32]
 *Remote Process Group*: Remote Process Groups appear and behave similar to Process Groups. However, the Remote Process Group (RPG)
 references a remote instance of NiFi. When an RPG is dragged onto the canvas, rather than being prompted for a name, the DFM
 is prompted for the URL of the remote NiFi instance. If the remote NiFi is a clustered instance, the URL that should be used
-is the URL of the remote instance's NiFi Cluster Manager (NCM). When data is transferred to a clustered instance of NiFi
-via an RPG, the RPG will first connect to the remote instance's NCM to determine which nodes are in the cluster and
-how busy each node is. This information is then used to load balance the data that is pushed to each node. The remote NCM is
+is the URL of any NiFi instance in that cluster. When data is transferred to a clustered instance of NiFi
+via an RPG, the RPG will first connect to the remote instance whose URL is configured to determine which nodes are in the cluster and
+how busy each node is. This information is then used to load balance the data that is pushed to each node. The remote instances are
 then interrogated periodically to determine information about any nodes that are dropped from or added to the cluster and to
 recalculate the load balancing based on each node's load. For more information, see the section on <<site-to-site,Site-to-Site>>.
 
@@ -995,7 +995,13 @@ link:administration-guide.html[System Administrator\u2019s Guide].
 
 image:configure-remote-process-group.png["Configure Remote Process Group"]
 
-By default, it is set to _RAW_ which uses raw socket communication using a dedicated port. _HTTP_ transport protocol is especially useful if the remote NiFi instance is in a restricted network that only allow access through HTTP(S) protocol or only accessible from a specific HTTP Proxy server. For accessing through a HTTP Proxy Server, BASIC and DIGEST authentication are supported.
+By default, it is set to _RAW_ which uses raw socket communication using a dedicated port. _HTTP_ transport protocol is especially useful if the remote
+NiFi instance is in a restricted network that only allow access through HTTP(S) protocol or only accessible from a specific HTTP Proxy server.
+For accessing through a HTTP Proxy Server, BASIC and DIGEST authentication are supported.
+
+*Local Network Interface*: In some cases, it may be desirable to prefer one network interface over another. For example, if a wired interface and a wireless
+interface both exist, the wired interface may be preferred. This can be configured by specifying the name of the network interface to use in this box. If the
+value entered is not valid, the Remote Process Group will not be valid and will not communicate with other NiFi instances until this is resolved.
 
 ==== Configure Site-to-Site server NiFi instance
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
index df01b82..ee64b69 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java
@@ -39,12 +39,14 @@ public class RemoteProcessGroupDTO extends ComponentDTO {
     private String communicationsTimeout;
     private String yieldDuration;
     private String transportProtocol;
+    private String localNetworkInterface;
     private String proxyHost;
     private Integer proxyPort;
     private String proxyUser;
     private String proxyPassword;
 
     private Collection<String> authorizationIssues;
+    private Collection<String> validationErrors;
     private Boolean transmitting;
 
     private Integer inputPortCount;
@@ -349,6 +351,25 @@ public class RemoteProcessGroupDTO extends ComponentDTO {
         this.transportProtocol = transportProtocol;
     }
 
+    @ApiModelProperty("The local network interface to send/receive data. If not specified, any local address is used. If clustered, all nodes must have an interface with this identifier.")
+    public String getLocalNetworkInterface() {
+        return localNetworkInterface;
+    }
+
+    public void setLocalNetworkInterface(String localNetworkInterface) {
+        this.localNetworkInterface = localNetworkInterface;
+    }
+
+    @ApiModelProperty(
+            "The validation errors for the remote process group. These validation errors represent the problems with the remote process group that must be resolved before it can transmit."
+    )
+    public Collection<String> getValidationErrors() {
+        return validationErrors;
+    }
+
+    public void setValidationErrors(Collection<String> validationErrors) {
+        this.validationErrors = validationErrors;
+    }
 
     public String getProxyHost() {
         return proxyHost;

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/RemoteProcessGroupEntityMerger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/RemoteProcessGroupEntityMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/RemoteProcessGroupEntityMerger.java
index 3209e02..a426d93 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/RemoteProcessGroupEntityMerger.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/RemoteProcessGroupEntityMerger.java
@@ -33,7 +33,6 @@ public class RemoteProcessGroupEntityMerger implements ComponentEntityMerger<Rem
     public void merge(RemoteProcessGroupEntity clientEntity, Map<NodeIdentifier, RemoteProcessGroupEntity> entityMap) {
         ComponentEntityMerger.super.merge(clientEntity, entityMap);
         for (Map.Entry<NodeIdentifier, RemoteProcessGroupEntity> entry : entityMap.entrySet()) {
-            final NodeIdentifier nodeId = entry.getKey();
             final RemoteProcessGroupEntity entityStatus = entry.getValue();
             if (entityStatus != clientEntity) {
                 mergeStatus(clientEntity.getStatus(), clientEntity.getPermissions().getCanRead(), entry.getValue().getStatus(), entry.getValue().getPermissions().getCanRead(), entry.getKey());
@@ -47,6 +46,7 @@ public class RemoteProcessGroupEntityMerger implements ComponentEntityMerger<Rem
      * @param clientEntity the entity being returned to the client
      * @param entityMap all node responses
      */
+    @Override
     public void mergeComponents(final RemoteProcessGroupEntity clientEntity, final Map<NodeIdentifier, RemoteProcessGroupEntity> entityMap) {
         final RemoteProcessGroupDTO clientDto = clientEntity.getComponent();
         final Map<NodeIdentifier, RemoteProcessGroupDTO> dtoMap = new HashMap<>();
@@ -75,6 +75,7 @@ public class RemoteProcessGroupEntityMerger implements ComponentEntityMerger<Rem
         final RemoteProcessGroupContentsDTO remoteProcessGroupContents = clientDto.getContents();
 
         final Map<String, Set<NodeIdentifier>> authorizationErrorMap = new HashMap<>();
+        final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
         Boolean mergedIsTargetSecure = null;
         final Set<RemoteProcessGroupPortDTO> mergedInputPorts = new HashSet<>();
         final Set<RemoteProcessGroupPortDTO> mergedOutputPorts = new HashSet<>();
@@ -88,6 +89,7 @@ public class RemoteProcessGroupEntityMerger implements ComponentEntityMerger<Rem
 
                 // merge the authorization errors
                 ErrorMerger.mergeErrors(authorizationErrorMap, nodeId, nodeRemoteProcessGroup.getAuthorizationIssues());
+                ErrorMerger.mergeErrors(validationErrorMap, nodeId, nodeRemoteProcessGroup.getValidationErrors());
 
                 // use the first target secure flag since they will all be the same
                 final Boolean nodeIsTargetSecure = nodeRemoteProcessGroup.isTargetSecure();
@@ -124,5 +126,6 @@ public class RemoteProcessGroupEntityMerger implements ComponentEntityMerger<Rem
 
         // set the merged the validation errors
         clientDto.setAuthorizationIssues(ErrorMerger.normalizedMergedErrors(authorizationErrorMap, dtoMap.size()));
+        clientDto.setValidationErrors(ErrorMerger.normalizedMergedErrors(validationErrorMap, dtoMap.size()));
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
index 4366068..69ae396 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/remote/protocol/ServerProtocol.java
@@ -145,8 +145,8 @@ public interface ServerProtocol extends VersionedRemoteResource {
             Peer peer,
             Optional<ClusterNodeInformation> clusterNodeInfo,
             String remoteInputHost,
-            int remoteInputPort,
-            int remoteInputHttpPort,
+            Integer remoteInputPort,
+            Integer remoteInputHttpPort,
             boolean isSiteToSiteSecure) throws IOException;
 
     void shutdown(Peer peer);

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
index d865475..db9e68b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -45,6 +45,7 @@ import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
 import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.cluster.protocol.DataFlow;
@@ -1126,6 +1127,12 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
                 remoteGroup.setProxyPassword(remoteGroupDto.getProxyPassword());
             }
 
+            if (StringUtils.isBlank(remoteGroupDto.getLocalNetworkInterface())) {
+                remoteGroup.setNetworkInterface(null);
+            } else {
+                remoteGroup.setNetworkInterface(remoteGroupDto.getLocalNetworkInterface());
+            }
+
             final Set<RemoteProcessGroupPortDescriptor> inputPorts = new HashSet<>();
             for (final Element portElement : getChildrenByTagName(remoteProcessGroupElement, "inputPort")) {
                 inputPorts.add(FlowFromDOMFactory.getRemoteProcessGroupPort(portElement));

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
index 6c39f16..f8d38bc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
@@ -262,6 +262,7 @@ public class FlowFromDOMFactory {
         dto.setProxyHost(getString(element, "proxyHost"));
         dto.setProxyPort(getOptionalInt(element, "proxyPort"));
         dto.setProxyUser(getString(element, "proxyUser"));
+        dto.setLocalNetworkInterface(getString(element, "networkInterface"));
 
         final String rawPassword = getString(element, "proxyPassword");
         final String proxyPassword = encryptor == null ? rawPassword : decrypt(rawPassword, encryptor);

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
index b8936ba..f6e3d2b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
@@ -69,7 +69,7 @@ import org.w3c.dom.Node;
  */
 public class StandardFlowSerializer implements FlowSerializer {
 
-    private static final String MAX_ENCODING_VERSION = "1.0";
+    private static final String MAX_ENCODING_VERSION = "1.1";
 
     private final StringEncryptor encryptor;
 
@@ -261,6 +261,9 @@ public class StandardFlowSerializer implements FlowSerializer {
             final String value = ENC_PREFIX + encryptor.encrypt(remoteRef.getProxyPassword()) + ENC_SUFFIX;
             addTextElement(element, "proxyPassword", value);
         }
+        if (remoteRef.getNetworkInterface() != null) {
+            addTextElement(element, "networkInterface", remoteRef.getNetworkInterface());
+        }
 
         for (final RemoteGroupPort port : remoteRef.getInputPorts()) {
             if (port.hasIncomingConnection()) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 3634850..3679b98 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -679,6 +679,7 @@ public class FingerprintFactory {
     private StringBuilder addRemoteProcessGroupFingerprint(final StringBuilder builder, final Element remoteProcessGroupElem) throws FingerprintException {
         appendFirstValue(builder, DomUtils.getChildNodesByTagName(remoteProcessGroupElem, "id"));
         appendFirstValue(builder, DomUtils.getChildNodesByTagName(remoteProcessGroupElem, "url"));
+        appendFirstValue(builder, DomUtils.getChildNodesByTagName(remoteProcessGroupElem, "networkInterface"));
 
         final NodeList inputPortList = DomUtils.getChildNodesByTagName(remoteProcessGroupElem, "inputPort");
         final NodeList outputPortList = DomUtils.getChildNodesByTagName(remoteProcessGroupElem, "outputPort");

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 67c8f11..286b2dc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -879,30 +879,33 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
         writeLock.lock();
         try {
             this.networkInterfaceName = interfaceName;
+            if (interfaceName == null) {
+                this.nicValidationResult = null;
+            } else {
+                try {
+                    final Enumeration<InetAddress> inetAddresses = NetworkInterface.getByName(interfaceName).getInetAddresses();
 
-            try {
-                final Enumeration<InetAddress> inetAddresses = NetworkInterface.getByName(interfaceName).getInetAddresses();
-
-                if (inetAddresses.hasMoreElements()) {
-                    this.localAddress = inetAddresses.nextElement();
-                    this.nicValidationResult = null;
-                } else {
+                    if (inetAddresses.hasMoreElements()) {
+                        this.localAddress = inetAddresses.nextElement();
+                        this.nicValidationResult = null;
+                    } else {
+                        this.localAddress = null;
+                        this.nicValidationResult = new ValidationResult.Builder()
+                            .input(interfaceName)
+                            .subject("Network Interface Name")
+                            .valid(false)
+                            .explanation("No IP Address could be found that is bound to the interface with name " + interfaceName)
+                            .build();
+                    }
+                } catch (final Exception e) {
                     this.localAddress = null;
                     this.nicValidationResult = new ValidationResult.Builder()
                         .input(interfaceName)
                         .subject("Network Interface Name")
                         .valid(false)
-                        .explanation("No IP Address could be found that is bound to the interface with name " + interfaceName)
+                        .explanation("Could not obtain Network Interface with name " + interfaceName)
                         .build();
                 }
-            } catch (final Exception e) {
-                this.localAddress = null;
-                this.nicValidationResult = new ValidationResult.Builder()
-                    .input(interfaceName)
-                    .subject("Network Interface Name")
-                    .valid(false)
-                    .explanation("Could not obtain Network Interface with name " + interfaceName)
-                    .build();
             }
         } finally {
             writeLock.unlock();

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
index 4607320..02a9ca5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
@@ -205,6 +205,7 @@
             <xs:element name="proxyPort" type="xs:int" minOccurs="0" maxOccurs="1" />
             <xs:element name="proxyUser" type="xs:string" minOccurs="0" maxOccurs="1" />
             <xs:element name="proxyPassword" type="xs:string" minOccurs="0" maxOccurs="1" />
+            <xs:element name="networkInterface" type="xs:string" minOccurs="0" maxOccurs="1" />
 
             <!-- The input ports and output ports of the remote group may change without our knowledge; however,
             they are persisted here because on a restart of NiFi, we need to have the Input & Output Ports' IDs

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java
index a35b160..57bebda 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/http/StandardHttpFlowFileServerProtocol.java
@@ -228,7 +228,7 @@ public class StandardHttpFlowFileServerProtocol extends AbstractFlowFileServerPr
     }
 
     @Override
-    public void sendPeerList(Peer peer, Optional<ClusterNodeInformation> clusterNodeInfo, String remoteInputHost, int remoteInputPort, int remoteInputHttpPort,
+    public void sendPeerList(Peer peer, Optional<ClusterNodeInformation> clusterNodeInfo, String remoteInputHost, Integer remoteInputPort, Integer remoteInputHttpPort,
                              boolean isSiteToSiteSecure) throws IOException {
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index b67276e..6c22ac7 100644
--- a/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@ -156,8 +156,8 @@ public class SocketFlowFileServerProtocol extends AbstractFlowFileServerProtocol
             final Peer peer,
             final Optional<ClusterNodeInformation> clusterNodeInfo,
             final String remoteInputHost,
-            final int remoteInputPort,
-            final int remoteInputHttpPort,
+        final Integer remoteInputPort,
+        final Integer remoteInputHttpPort,
             final boolean isSiteToSiteSecure) throws IOException {
         if (!handshakeCompleted) {
             throw new IllegalStateException("Handshake has not been completed");

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 113d491..f5c1b88 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -18,6 +18,7 @@ package org.apache.nifi.web.api.dto;
 
 import java.text.Collator;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -1550,21 +1551,21 @@ public final class DtoFactory {
         }
 
         if (group.getAuthorizationIssue() != null) {
-            final List<String> authIssues = new ArrayList<>();
-            final String authIssue = group.getAuthorizationIssue();
-            if (authIssue != null) {
-                authIssues.add(authIssue);
-            }
+            dto.setAuthorizationIssues(Arrays.asList(group.getAuthorizationIssue()));
+        }
 
-            final Collection<ValidationResult> validationResults = group.validate();
-            validationResults.stream()
-                .filter(result -> !result.isValid())
-                .map(result -> result.toString())
-                .forEach(str -> authIssues.add(str));
+        final Collection<ValidationResult> validationErrors = group.validate();
+        if (validationErrors != null && !validationErrors.isEmpty()) {
+            final List<String> errors = new ArrayList<>();
+            for (final ValidationResult validationResult : validationErrors) {
+                errors.add(validationResult.toString());
+            }
 
-            dto.setAuthorizationIssues(authIssues);
+            dto.setValidationErrors(errors);
         }
 
+        dto.setLocalNetworkInterface(group.getNetworkInterface());
+
         dto.setActiveRemoteInputPortCount(activeRemoteInputPortCount);
         dto.setInactiveRemoteInputPortCount(inactiveRemoteInputPortCount);
         dto.setActiveRemoteOutputPortCount(activeRemoteOutputPortCount);

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardRemoteProcessGroupDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardRemoteProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardRemoteProcessGroupDAO.java
index d022b15..a93c410 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardRemoteProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardRemoteProcessGroupDAO.java
@@ -16,6 +16,14 @@
  */
 package org.apache.nifi.web.dao.impl;
 
+import static org.apache.nifi.util.StringUtils.isEmpty;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.connectable.Position;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.exception.ValidationException;
@@ -32,13 +40,6 @@ import org.apache.nifi.web.dao.RemoteProcessGroupDAO;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Matcher;
-
-import static org.apache.nifi.util.StringUtils.isEmpty;
-
 public class StandardRemoteProcessGroupDAO extends ComponentDAO implements RemoteProcessGroupDAO {
 
     private static final Logger logger = LoggerFactory.getLogger(StandardRemoteProcessGroupDAO.class);
@@ -144,6 +145,7 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
 
         // if any remote group properties are changing, verify update
         if (isAnyNotNull(remoteProcessGroupDto.getYieldDuration(),
+                remoteProcessGroupDto.getLocalNetworkInterface(),
                 remoteProcessGroupDto.getCommunicationsTimeout(),
                 remoteProcessGroupDto.getProxyHost(),
                 remoteProcessGroupDto.getProxyPort(),
@@ -359,6 +361,7 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
         final String proxyPassword = remoteProcessGroupDTO.getProxyPassword();
 
         final String transportProtocol = remoteProcessGroupDTO.getTransportProtocol();
+        final String localNetworkInterface = remoteProcessGroupDTO.getLocalNetworkInterface();
 
         if (isNotNull(name)) {
             remoteProcessGroup.setName(name);
@@ -391,6 +394,13 @@ public class StandardRemoteProcessGroupDAO extends ComponentDAO implements Remot
                 remoteProcessGroup.setProxyPassword(proxyPassword);
             }
         }
+        if (localNetworkInterface != null) {
+            if (StringUtils.isBlank(localNetworkInterface)) {
+                remoteProcessGroup.setNetworkInterface(null);
+            } else {
+                remoteProcessGroup.setNetworkInterface(localNetworkInterface);
+            }
+        }
 
         final Boolean isTransmitting = remoteProcessGroupDTO.isTransmitting();
         if (isNotNull(isTransmitting)) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/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-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-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 4af046e..4948001 100644
--- a/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-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
@@ -26,16 +26,28 @@
             </div>
         </div>
         <div class="setting">
-            <div class="setting-name">
-                Transport Protocol
-                <div class="fa fa-question-circle" alt="Info" title="Specify the transport protocol to use for this Remote Process Group communication."></div>
+            <div class="remote-process-group-setting-left">
+                <div class="setting-name">
+                    Transport Protocol
+                    <div class="fa fa-question-circle" alt="Info" title="Specify the transport protocol to use for this Remote Process Group communication."></div>
+                </div>
+                <div class="setting-field">
+                    <div id="new-remote-process-group-transport-protocol-combo"></div>
+                </div>
             </div>
-            <div class="setting-field">
-                <div id="new-remote-process-group-transport-protocol-combo"></div>
+            <div class="remote-process-group-setting-right">
+                <div class="setting-name">
+                    Local Network Interface
+                    <div class="fa fa-question-circle" alt="Info" title="The local network interface to send/receive data. If not specified, any local address is used. If clustered, all nodes must have an interface with this identifier."></div>
+                </div>
+                <div class="setting-field">
+                    <input type="text" class="small-setting-input" id="new-remote-process-group-local-network-interface"/>
+                </div>
             </div>
+            <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-proxy-host-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     HTTP Proxy server hostname
                     <div class="fa fa-question-circle" alt="Info" title="Specify the proxy server's hostname to use. If not specified, HTTP traffics are sent directly to the target NiFi instance."></div>
@@ -44,7 +56,7 @@
                     <input type="text" class="small-setting-input" id="new-remote-process-group-proxy-host"/>
                 </div>
             </div>
-            <div class="remote-process-group-proxy-port-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     HTTP Proxy server port
                     <div class="fa fa-question-circle" alt="Info" title="Specify the proxy server's port number, optional. If not specified, default port 80 will be used."></div>
@@ -56,7 +68,7 @@
             <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-proxy-user-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     HTTP Proxy user
                     <div class="fa fa-question-circle" alt="Info" title="Specify an user name to connect to the proxy server, optional."></div>
@@ -65,7 +77,7 @@
                     <input type="text" class="small-setting-input" id="new-remote-process-group-proxy-user"/>
                 </div>
             </div>
-            <div class="remote-process-group-proxy-password-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     HTTP Proxy password
                     <div class="fa fa-question-circle" alt="Info" title="Specify an user password to connect to the proxy server, optional."></div>
@@ -77,7 +89,7 @@
             <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-timeout-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     Communications timeout
                     <div class="fa fa-question-circle" alt="Info" title="When communication with this remote process group takes longer than this amount of time, it will timeout."></div>
@@ -86,7 +98,7 @@
                     <input type="text" class="small-setting-input" id="new-remote-process-group-timeout"/>
                 </div>
             </div>
-            <div class="remote-process-group-yield-duration-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     Yield duration
                     <div class="fa fa-question-circle" alt="Info" title="When communication with this remote process group fails, it will not be scheduled again until this amount of time elapses."></div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-configuration.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-configuration.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-configuration.jsp
index c8af6d3..36ecd48 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-configuration.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-configuration.jsp
@@ -36,16 +36,28 @@
             </div>
         </div>
         <div class="setting">
-            <div class="setting-name">
-                Transport Protocol
-                <div class="fa fa-question-circle" alt="Info" title="Specify the transport protocol to use for this Remote Process Group communication."></div>
+            <div class="remote-process-group-setting-left">
+                <div class="setting-name">
+                    Transport Protocol
+                    <div class="fa fa-question-circle" alt="Info" title="Specify the transport protocol to use for this Remote Process Group communication."></div>
+                </div>
+                <div class="setting-field">
+                    <div id="remote-process-group-transport-protocol-combo"></div>
+                </div>
             </div>
-            <div class="setting-field">
-                <div id="remote-process-group-transport-protocol-combo"></div>
+            <div class="remote-process-group-setting-right">
+                <div class="setting-name">
+                    Local Network Interface
+                    <div class="fa fa-question-circle" alt="Info" title="The local network interface to send/receive data. If not specified, any local address is used. If clustered, all nodes must have an interface with this identifier."></div>
+                </div>
+                <div class="setting-field">
+                    <input type="text" class="small-setting-input" id="remote-process-group-local-network-interface"/>
+                </div>
             </div>
+            <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-proxy-host-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     HTTP Proxy server hostname
                     <div class="fa fa-question-circle" alt="Info" title="Specify the proxy server's hostname to use. If not specified, HTTP traffics are sent directly to the target NiFi instance."></div>
@@ -54,7 +66,7 @@
                     <input type="text" class="small-setting-input" id="remote-process-group-proxy-host"/>
                 </div>
             </div>
-            <div class="remote-process-group-proxy-port-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     HTTP Proxy server port
                     <div class="fa fa-question-circle" alt="Info" title="Specify the proxy server's port number, optional. If not specified, default port 80 will be used."></div>
@@ -66,7 +78,7 @@
             <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-proxy-user-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     HTTP Proxy user
                     <div class="fa fa-question-circle" alt="Info" title="Specify an user name to connect to the proxy server, optional."></div>
@@ -75,7 +87,7 @@
                     <input type="text" class="small-setting-input" id="remote-process-group-proxy-user"/>
                 </div>
             </div>
-            <div class="remote-process-group-proxy-password-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     HTTP Proxy password
                     <div class="fa fa-question-circle" alt="Info" title="Specify an user password to connect to the proxy server, optional."></div>
@@ -87,7 +99,7 @@
             <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-timeout-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     Communications timeout
                     <div class="fa fa-question-circle" alt="Info" title="When communication with this remote process group takes longer than this amount of time, it will timeout."></div>
@@ -96,7 +108,7 @@
                     <input type="text" class="small-setting-input" id="remote-process-group-timeout"/>
                 </div>
             </div>
-            <div class="remote-process-group-yield-duration-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     Yield duration
                     <div class="fa fa-question-circle" alt="Info" title="When communication with this remote process group fails, it will not be scheduled again until this amount of time elapses."></div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-details.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-details.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-details.jsp
index 6f7f992..3388a68 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-details.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/remote-process-group-details.jsp
@@ -36,16 +36,28 @@
             </div>
         </div>
         <div class="setting">
-            <div class="setting-name">
-                Transport Protocol
-                <div class="fa fa-question-circle" alt="Info" title="Transport protocol to use for this Remote Process Group communication."></div>
+            <div class="remote-process-group-setting-left">
+                <div class="setting-name">
+                    Transport Protocol
+                    <div class="fa fa-question-circle" alt="Info" title="Transport protocol to use for this Remote Process Group communication."></div>
+                </div>
+                <div class="setting-field">
+                    <div id="read-only-remote-process-group-transport-protocol"></div>
+                </div>
             </div>
-            <div class="setting-field">
-                <div id="read-only-remote-process-group-transport-protocol"></div>
+            <div class="remote-process-group-setting-right">
+                <div class="setting-name">
+                    Local Network Interface
+                    <div class="fa fa-question-circle" alt="Info" title="The local network interface to send/receive data. If not specified, any local address is used. If clustered, all nodes must have an interface with this identifier."></div>
+                </div>
+                <div class="setting-field">
+                    <span id="read-only-remote-process-group-local-network-interface"></span>
+                </div>
             </div>
+            <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-proxy-host-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     HTTP Proxy server hostname
                     <div class="fa fa-question-circle" alt="Info" title="Specify the proxy server's hostname to use. If not specified, HTTP traffics are sent directly to the target NiFi instance."></div>
@@ -54,7 +66,7 @@
                     <span id="read-only-remote-process-group-proxy-host"></span>
                 </div>
             </div>
-            <div class="remote-process-group-proxy-port-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     HTTP Proxy server port
                     <div class="fa fa-question-circle" alt="Info" title="Specify the proxy server's port number, optional. If not specified, default port 80 will be used."></div>
@@ -66,7 +78,7 @@
             <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-proxy-user-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     HTTP Proxy user
                     <div class="fa fa-question-circle" alt="Info" title="Specify an user name to connect to the proxy server, optional."></div>
@@ -75,7 +87,7 @@
                     <span id="read-only-remote-process-group-proxy-user"></span>
                 </div>
             </div>
-            <div class="remote-process-group-proxy-password-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     HTTP Proxy password
                     <div class="fa fa-question-circle" alt="Info" title="Specify an user password to connect to the proxy server, optional."></div>
@@ -87,7 +99,7 @@
             <div class="clear"></div>
         </div>
         <div class="setting">
-            <div class="remote-process-group-timeout-setting">
+            <div class="remote-process-group-setting-left">
                 <div class="setting-name">
                     Communications timeout
                     <div class="fa fa-question-circle" alt="Info" title="When communication with this remote process group takes longer than this amount of time, it will timeout."></div>
@@ -96,7 +108,7 @@
                     <span id="read-only-remote-process-group-timeout"></span>
                 </div>
             </div>
-            <div class="remote-process-group-yield-duration-setting">
+            <div class="remote-process-group-setting-right">
                 <div class="setting-name">
                     Yield duration
                     <div class="fa fa-question-circle" alt="Info" title="When communication with this remote process group fails, it will not be scheduled again until this amount of time elapses."></div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
index 40a520f..4aaa253 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css
@@ -18,10 +18,6 @@
     Specific dialog settings.
 */
 
-#new-remote-process-group-transport-protocol-combo {
-    width: 160px;
-}
-
 #fill-color-dialog {
     display: none;
     width: 240px;

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/remote-process-group-configuration.css
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/remote-process-group-configuration.css b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/remote-process-group-configuration.css
index cc235b6..58dcfcd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/remote-process-group-configuration.css
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/remote-process-group-configuration.css
@@ -49,22 +49,13 @@
     display: none;
 }
 
-#remote-process-group-transport-protocol-combo {
-    width: 160px;
-}
-
-div.remote-process-group-timeout-setting, div.remote-process-group-yield-duration-setting,
-div.remote-process-group-proxy-host-setting, div.remote-process-group-proxy-port-setting,
-div.remote-process-group-proxy-user-setting, div.remote-process-group-proxy-password-setting {
+div.remote-process-group-setting-left, div.remote-process-group-setting-right {
     float: left;
     width: 49%;
 }
 
-div.remote-process-group-yield-duration-setting,
-div.remote-process-group-proxy-port-setting,
-div.remote-process-group-proxy-password-setting {
+div.remote-process-group-setting-right {
     margin-left: 2%;
-    width: 49%;
 }
 
 /* remote process group port configuration */

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-remote-process-group-component.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-remote-process-group-component.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-remote-process-group-component.js
index 5e6e116..08f1b1a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-remote-process-group-component.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/header/components/nf-ng-remote-process-group-component.js
@@ -78,6 +78,7 @@
                     'communicationsTimeout': $('#new-remote-process-group-timeout').val(),
                     'yieldDuration': $('#new-remote-process-group-yield-duration').val(),
                     'transportProtocol': $('#new-remote-process-group-transport-protocol-combo').combo('getSelectedOption').value,
+                    'localNetworkInterface': $('#new-remote-process-group-local-network-interface').val(),
                     'proxyHost': $('#new-remote-process-group-proxy-host').val(),
                     'proxyPort': $('#new-remote-process-group-proxy-port').val(),
                     'proxyUser': $('#new-remote-process-group-proxy-user').val(),
@@ -155,6 +156,7 @@
                 init: function () {
                     var defaultTimeout = "30 sec";
                     var defaultYieldDuration = "10 sec";
+
                     // configure the new remote process group dialog
                     this.getElement().modal({
                         scrollableContentStyle: 'scrollable',
@@ -167,6 +169,7 @@
                                 $('#new-remote-process-group-transport-protocol-combo').combo('setSelectedOption', {
                                     value: 'RAW'
                                 });
+                                $('#new-remote-process-group-local-network-interface').val('');
                                 $('#new-remote-process-group-proxy-host').val('');
                                 $('#new-remote-process-group-proxy-port').val('');
                                 $('#new-remote-process-group-proxy-user').val('');
@@ -174,9 +177,11 @@
                             }
                         }
                     });
+
                     // set default values
                     $('#new-remote-process-group-timeout').val(defaultTimeout);
                     $('#new-remote-process-group-yield-duration').val(defaultYieldDuration);
+
                     // initialize the transport protocol combo
                     $('#new-remote-process-group-transport-protocol-combo').combo({
                         options: [{

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-configuration.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-configuration.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-configuration.js
index 06cf866..2712713 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-configuration.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-configuration.js
@@ -84,7 +84,8 @@
                                     proxyHost: $('#remote-process-group-proxy-host').val(),
                                     proxyPort: $('#remote-process-group-proxy-port').val(),
                                     proxyUser: $('#remote-process-group-proxy-user').val(),
-                                    proxyPassword: $('#remote-process-group-proxy-password').val()
+                                    proxyPassword: $('#remote-process-group-proxy-password').val(),
+                                    localNetworkInterface: $('#remote-process-group-local-network-interface').val()
                                 }
                             };
 
@@ -150,6 +151,7 @@
                         $('#remote-process-group-transport-protocol-combo').combo('setSelectedOption', {
                             value: 'RAW'
                         });
+                        $('#remote-process-group-local-network-interface').val('');
                         $('#remote-process-group-proxy-host').val('');
                         $('#remote-process-group-proxy-port').val('');
                         $('#remote-process-group-proxy-user').val('');
@@ -191,6 +193,7 @@
                 $('#remote-process-group-proxy-port').val(selectionData.component.proxyPort);
                 $('#remote-process-group-proxy-user').val(selectionData.component.proxyUser);
                 $('#remote-process-group-proxy-password').val(selectionData.component.proxyPassword);
+                $('#remote-process-group-local-network-interface').val(selectionData.component.localNetworkInterface);
 
                 // select the appropriate transport-protocol
                 $('#remote-process-group-transport-protocol-combo').combo('setSelectedOption', {

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-details.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-details.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-details.js
index 7c6d667..6c61398 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-details.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group-details.js
@@ -65,6 +65,7 @@
                         nfCommon.clearField('read-only-remote-process-group-timeout');
                         nfCommon.clearField('read-only-remote-process-group-yield-duration');
                         nfCommon.clearField('read-only-remote-process-group-transport-protocol');
+                        nfCommon.clearField('read-only-remote-process-group-local-network-interface');
                         nfCommon.clearField('read-only-remote-process-group-proxy-host');
                         nfCommon.clearField('read-only-remote-process-group-proxy-port');
                         nfCommon.clearField('read-only-remote-process-group-proxy-user');
@@ -91,6 +92,7 @@
                 nfCommon.populateField('read-only-remote-process-group-timeout', selectionData.component.communicationsTimeout);
                 nfCommon.populateField('read-only-remote-process-group-yield-duration', selectionData.component.yieldDuration);
                 nfCommon.populateField('read-only-remote-process-group-transport-protocol', selectionData.component.transportProtocol);
+                nfCommon.populateField('read-only-remote-process-group-local-network-interface', selectionData.component.localNetworkInterface);
                 nfCommon.populateField('read-only-remote-process-group-proxy-host', selectionData.component.proxyHost);
                 nfCommon.populateField('read-only-remote-process-group-proxy-port', selectionData.component.proxyPort);
                 nfCommon.populateField('read-only-remote-process-group-proxy-user', selectionData.component.proxyUser);

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
index 9842c71..b91afd1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-remote-process-group.js
@@ -667,6 +667,21 @@
         });
     };
 
+    var hasIssues = function (d) {
+        return !nfCommon.isEmpty(d.component.authorizationIssues) || !nfCommon.isEmpty(d.component.validationErrors);
+    };
+
+    var getIssues = function (d) {
+        var issues = [];
+        if (!nfCommon.isEmpty(d.component.authorizationIssues)) {
+            issues = issues.concat(d.component.authorizationIssues);
+        }
+        if (!nfCommon.isEmpty(d.component.validationErrors)) {
+            issues = issues.concat(d.component.validationErrors);
+        }
+        return issues;
+    };
+
     /**
      * Updates the process group status.
      *
@@ -723,7 +738,7 @@
             .text(function (d) {
                 var icon = '';
                 if (d.permissions.canRead) {
-                    if (!nfCommon.isEmpty(d.component.authorizationIssues)) {
+                    if (hasIssues(d)) {
                         icon = '\uf071';
                     } else if (d.component.transmitting === true) {
                         icon = '\uf140';
@@ -736,7 +751,7 @@
             .attr('font-family', function (d) {
                 var family = '';
                 if (d.permissions.canRead) {
-                    if (!nfCommon.isEmpty(d.component.authorizationIssues) || d.component.transmitting) {
+                    if (hasIssues(d) || d.component.transmitting) {
                         family = 'FontAwesome';
                     } else {
                         family = 'flowfont';
@@ -745,20 +760,20 @@
                 return family;
             })
             .classed('invalid', function (d) {
-                return d.permissions.canRead && !nfCommon.isEmpty(d.component.authorizationIssues);
+                return d.permissions.canRead && hasIssues(d);
             })
             .classed('transmitting', function (d) {
-                return d.permissions.canRead && nfCommon.isEmpty(d.component.authorizationIssues) && d.component.transmitting === true;
+                return d.permissions.canRead && !hasIssues(d) && d.component.transmitting === true;
             })
             .classed('not-transmitting', function (d) {
-                return d.permissions.canRead && nfCommon.isEmpty(d.component.authorizationIssues) && d.component.transmitting === false;
+                return d.permissions.canRead && !hasIssues(d) && d.component.transmitting === false;
             })
             .each(function (d) {
                 // get the tip
                 var tip = d3.select('#authorization-issues-' + d.id);
 
                 // if there are validation errors generate a tooltip
-                if (d.permissions.canRead && !nfCommon.isEmpty(d.component.authorizationIssues)) {
+                if (d.permissions.canRead && hasIssues(d)) {
                     // create the tip if necessary
                     if (tip.empty()) {
                         tip = d3.select('#remote-process-group-tooltips').append('div')
@@ -770,7 +785,7 @@
 
                     // update the tip
                     tip.html(function () {
-                        var list = nfCommon.formatUnorderedList(d.component.authorizationIssues);
+                        var list = nfCommon.formatUnorderedList(getIssues(d));
                         if (list === null || list.length === 0) {
                             return '';
                         } else {

http://git-wip-us.apache.org/repos/asf/nifi/blob/16bde02e/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java b/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java
index 7576c5f..7808d21 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java
@@ -45,7 +45,7 @@ public class SSLCommsSession implements CommsSession {
     private int protocolVersion;
 
     public SSLCommsSession(final SSLContext sslContext, final String hostname, final int port) throws IOException {
-        sslSocketChannel = new SSLSocketChannel(sslContext, hostname, port, true);
+        sslSocketChannel = new SSLSocketChannel(sslContext, hostname, port, null, true);
 
         in = new SSLSocketChannelInputStream(sslSocketChannel);
         bufferedIn = new BufferedInputStream(in);