You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2018/08/31 15:08:01 UTC

[2/2] nifi git commit: NIFI-5282: Add ProxyConfigurationService to GCSProcessors

NIFI-5282: Add ProxyConfigurationService to GCSProcessors

This closes #2981


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

Branch: refs/heads/master
Commit: 5a58c9a1715df1da32f053ac8af5c73c593a569e
Parents: cdae2b1
Author: Koji Kawamura <ij...@apache.org>
Authored: Fri Aug 31 18:47:59 2018 +0900
Committer: Matt Gilman <ma...@gmail.com>
Committed: Fri Aug 31 11:07:09 2018 -0400

----------------------------------------------------------------------
 .../nifi-gcp-bundle/nifi-gcp-processors/pom.xml |  4 +
 .../processors/gcp/AbstractGCPProcessor.java    | 14 ++--
 .../gcp/ProxyAwareTransportFactory.java         | 78 ++++++++++++++++++++
 .../factory/CredentialPropertyDescriptors.java  | 17 -----
 .../credentials/factory/CredentialsFactory.java |  5 +-
 .../factory/CredentialsStrategy.java            |  6 +-
 .../strategies/AbstractCredentialsStrategy.java |  5 --
 ...stractServiceAccountCredentialsStrategy.java | 29 +-------
 .../ComputeEngineCredentialsStrategy.java       |  7 +-
 ...itApplicationDefaultCredentialsStrategy.java |  5 +-
 ...itApplicationDefaultCredentialsStrategy.java |  5 +-
 .../GCPCredentialsControllerService.java        | 19 +++--
 .../gcp/storage/AbstractGCSProcessor.java       | 71 ++++++++++--------
 .../factory/CredentialsFactoryTest.java         | 16 ++--
 .../nifi-gcp-services-api-nar/pom.xml           |  6 ++
 15 files changed, 181 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
index af9baa4..ddf7ef9 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
@@ -36,6 +36,10 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-proxy-configuration-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
             <version>1.8.0-SNAPSHOT</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
index f5cd488..2c178ff 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java
@@ -27,6 +27,7 @@ import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.proxy.ProxyConfiguration;
 
 import java.util.List;
 
@@ -76,16 +77,18 @@ public abstract class AbstractGCPProcessor<
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor HTTP_PROXY_USERNAME = new PropertyDescriptor.Builder()
-            .name("Http Proxy Username")
+    public static final PropertyDescriptor HTTP_PROXY_USERNAME = new PropertyDescriptor
+            .Builder().name("gcp-proxy-user-name")
+            .displayName("Http Proxy Username")
             .description("Http Proxy Username")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .required(false)
             .build();
 
-    public static final PropertyDescriptor HTTP_PROXY_PASSWORD = new PropertyDescriptor.Builder()
-            .name("Http Proxy Password")
+    public static final PropertyDescriptor HTTP_PROXY_PASSWORD = new PropertyDescriptor
+            .Builder().name("gcp-proxy-user-password")
+            .displayName("Http Proxy Password")
             .description("Http Proxy Password")
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
@@ -120,7 +123,8 @@ public abstract class AbstractGCPProcessor<
                 PROXY_HOST,
                 PROXY_PORT,
                 HTTP_PROXY_USERNAME,
-                HTTP_PROXY_PASSWORD
+                HTTP_PROXY_PASSWORD,
+                ProxyConfiguration.createProxyConfigPropertyDescriptor(true, ProxyAwareTransportFactory.PROXY_SPECS)
         );
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/ProxyAwareTransportFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/ProxyAwareTransportFactory.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/ProxyAwareTransportFactory.java
new file mode 100644
index 0000000..4852416
--- /dev/null
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/ProxyAwareTransportFactory.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.gcp;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.http.apache.ApacheHttpTransport;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.auth.http.HttpTransportFactory;
+import org.apache.http.HttpHost;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.conn.params.ConnRouteParams;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.net.Proxy;
+
+public class ProxyAwareTransportFactory implements HttpTransportFactory {
+
+    private static final HttpTransport DEFAULT_TRANSPORT = new NetHttpTransport();
+    public static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
+
+    private final ProxyConfiguration proxyConfig;
+
+    public ProxyAwareTransportFactory(ProxyConfiguration proxyConfig) {
+        this.proxyConfig = proxyConfig;
+    }
+
+    @Override
+    public HttpTransport create() {
+
+        if (proxyConfig == null) {
+            return DEFAULT_TRANSPORT;
+        }
+
+        final Proxy proxy = proxyConfig.createProxy();
+
+        if (Proxy.Type.HTTP.equals(proxy.type()) && proxyConfig.hasCredential()) {
+            // If it requires authentication via username and password, use ApacheHttpTransport
+            final String host = proxyConfig.getProxyServerHost();
+            final int port = proxyConfig.getProxyServerPort();
+            final HttpHost proxyHost = new HttpHost(host, port);
+
+            final DefaultHttpClient httpClient = new DefaultHttpClient();
+            ConnRouteParams.setDefaultProxy(httpClient.getParams(), proxyHost);
+
+            if (proxyConfig.hasCredential()) {
+                final AuthScope proxyAuthScope = new AuthScope(host, port);
+                final UsernamePasswordCredentials proxyCredential
+                        = new UsernamePasswordCredentials(proxyConfig.getProxyUserName(), proxyConfig.getProxyUserPassword());
+                final BasicCredentialsProvider credentialsProvider = new BasicCredentialsProvider();
+                credentialsProvider.setCredentials(proxyAuthScope, proxyCredential);
+                httpClient.setCredentialsProvider(credentialsProvider);
+            }
+
+            return new ApacheHttpTransport(httpClient);
+
+        }
+
+        return new NetHttpTransport.Builder().setProxy(proxy).build();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
index a2218ea..9d3a519 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java
@@ -89,21 +89,4 @@ public final class CredentialPropertyDescriptors {
             .sensitive(true)
             .build();
 
-    public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor
-            .Builder().name("gcp-proxy-host")
-            .displayName("Proxy host")
-            .description("IP or hostname of the proxy to be used")
-            .required(false)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROXY_PORT = new PropertyDescriptor
-            .Builder().name("gcp-proxy-port")
-            .displayName("Proxy port")
-            .description("Proxy port number")
-            .required(false)
-            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
-            .addValidator(StandardValidators.INTEGER_VALIDATOR)
-            .build();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.java
index 8b9c4f4..f4d54c2 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory;
 
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
@@ -97,8 +98,8 @@ public class CredentialsFactory {
      *
      * @throws IOException if there is an issue accessing the credential files
      */
-    public GoogleCredentials getGoogleCredentials(final Map<PropertyDescriptor, String> properties) throws IOException {
+    public GoogleCredentials getGoogleCredentials(final Map<PropertyDescriptor, String> properties, final HttpTransportFactory transportFactory) throws IOException {
         final CredentialsStrategy primaryStrategy = selectPrimaryStrategy(properties);
-        return primaryStrategy.getGoogleCredentials(properties);
+        return primaryStrategy.getGoogleCredentials(properties, transportFactory);
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.java
index b2a9168..1765e14 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory;
 
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
@@ -56,8 +57,9 @@ public interface CredentialsStrategy {
 
     /**
      * Creates an AuthCredentials instance for this strategy, given the properties defined by the user.
-     *
+     * @param transportFactory Sub-classes should utilize this transport factory
+     *                        to support common network related configs such as proxy
      * @throws IOException if the provided credentials cannot be accessed or are invalid
      */
-    GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties) throws IOException;
+    GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties, HttpTransportFactory transportFactory) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.java
index 35b365e..50432d1 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.java
@@ -16,13 +16,11 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory.strategies;
 
-import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.processors.gcp.credentials.factory.CredentialsStrategy;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
@@ -79,9 +77,6 @@ public abstract class AbstractCredentialsStrategy implements CredentialsStrategy
         return validationFailureResults;
     }
 
-    @Override
-    public abstract GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties) throws IOException;
-
     public String getName() {
         return name;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.java
index d4547f5..a5d93ea 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.java
@@ -16,18 +16,12 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory.strategies;
 
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.components.PropertyDescriptor;
-import com.google.api.client.http.HttpTransport;
-import com.google.api.client.http.javanet.NetHttpTransport;
-import com.google.auth.http.HttpTransportFactory;
-import org.apache.nifi.util.StringUtils;
-import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.InetSocketAddress;
-import java.net.Proxy;
 import java.util.Map;
 
 /**
@@ -42,25 +36,8 @@ public abstract class AbstractServiceAccountCredentialsStrategy extends Abstract
     protected abstract InputStream getServiceAccountJson(Map<PropertyDescriptor, String> properties) throws IOException;
 
     @Override
-    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties) throws IOException {
-        final String proxyHost = properties.get(CredentialPropertyDescriptors.PROXY_HOST);
-        final String proxyPortString = properties.get(CredentialPropertyDescriptors.PROXY_PORT);
-        final Integer proxyPort = (proxyPortString != null && proxyPortString.matches("-?\\d+")) ?
-                Integer.parseInt(proxyPortString) : 0;
-
-        if (!StringUtils.isBlank(proxyHost) && proxyPort > 0) {
-            return GoogleCredentials.fromStream(getServiceAccountJson(properties),
-                    new HttpTransportFactory() {
-                        @Override
-                        public HttpTransport create() {
-                            return new NetHttpTransport.Builder()
-                                    .setProxy(new Proxy(Proxy.Type.HTTP, new InetSocketAddress(proxyHost, proxyPort)))
-                                    .build();
-                        }
-                    });
-        } else {
-            return GoogleCredentials.fromStream(getServiceAccountJson(properties));
-        }
+    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties, HttpTransportFactory transportFactory) throws IOException {
+            return GoogleCredentials.fromStream(getServiceAccountJson(properties), transportFactory);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.java
index 5b5034b..e98fb33 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory.strategies;
 
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.ComputeEngineCredentials;
 import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -36,7 +37,9 @@ public class ComputeEngineCredentialsStrategy extends AbstractBooleanCredentials
     }
 
     @Override
-    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties) throws IOException {
-        return new ComputeEngineCredentials();
+    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties, HttpTransportFactory transportFactory) throws IOException {
+        return ComputeEngineCredentials.newBuilder()
+                .setHttpTransportFactory(transportFactory)
+                .build();
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.java
index 55ebef7..213d0b7 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory.strategies;
 
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors;
@@ -38,8 +39,8 @@ public class ExplicitApplicationDefaultCredentialsStrategy extends AbstractBoole
     }
 
     @Override
-    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties) throws IOException {
-        return GoogleCredentials.getApplicationDefault();
+    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties, HttpTransportFactory transportFactory) throws IOException {
+        return GoogleCredentials.getApplicationDefault(transportFactory);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.java
index 95f9792..e0535b0 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory.strategies;
 
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.components.PropertyDescriptor;
 
@@ -35,8 +36,8 @@ public class ImplicitApplicationDefaultCredentialsStrategy extends AbstractCrede
     }
 
     @Override
-    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties) throws IOException {
-        return GoogleCredentials.getApplicationDefault();
+    public GoogleCredentials getGoogleCredentials(Map<PropertyDescriptor, String> properties, HttpTransportFactory transportFactory) throws IOException {
+        return GoogleCredentials.getApplicationDefault(transportFactory);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.java
index b7a4c05..ce5ada6 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.gcp.credentials.service;
 
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.GoogleCredentials;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
@@ -25,10 +26,12 @@ import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
 import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
 import org.apache.nifi.processors.gcp.credentials.factory.CredentialsFactory;
+import org.apache.nifi.proxy.ProxyConfiguration;
 import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -40,8 +43,7 @@ import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPrope
 import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE;
 import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_APPLICATION_DEFAULT_CREDENTIALS;
 import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_COMPUTE_ENGINE_CREDENTIALS;
-import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.PROXY_HOST;
-import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.PROXY_PORT;
+
 /**
  * Implementation of GCPCredentialsService interface
  *
@@ -63,8 +65,7 @@ public class GCPCredentialsControllerService extends AbstractControllerService i
         props.add(USE_COMPUTE_ENGINE_CREDENTIALS);
         props.add(SERVICE_ACCOUNT_JSON_FILE);
         props.add(SERVICE_ACCOUNT_JSON);
-        props.add(PROXY_HOST);
-        props.add(PROXY_PORT);
+        props.add(ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxyAwareTransportFactory.PROXY_SPECS));
         properties = Collections.unmodifiableList(props);
     }
 
@@ -82,13 +83,17 @@ public class GCPCredentialsControllerService extends AbstractControllerService i
 
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
-        return credentialsProviderFactory.validate(validationContext);
+        final Collection<ValidationResult> results = credentialsProviderFactory.validate(validationContext);
+        ProxyConfiguration.validateProxySpec(validationContext, results, ProxyAwareTransportFactory.PROXY_SPECS);
+        return results;
     }
 
     @OnEnabled
     public void onConfigured(final ConfigurationContext context) throws InitializationException {
         try {
-            googleCredentials = credentialsProviderFactory.getGoogleCredentials(context.getProperties());
+            final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+            final HttpTransportFactory transportFactory = new ProxyAwareTransportFactory(proxyConfiguration);
+            googleCredentials = credentialsProviderFactory.getGoogleCredentials(context.getProperties(), transportFactory);
         } catch (IOException e) {
             throw new InitializationException(e);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.java
index bf22faf..4363115 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.java
@@ -16,26 +16,25 @@
  */
 package org.apache.nifi.processors.gcp.storage;
 
-import com.google.api.client.http.HttpTransport;
-import com.google.api.client.http.javanet.NetHttpTransport;
 import com.google.api.gax.retrying.RetrySettings;
-import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.GoogleCredentials;
 import com.google.cloud.http.HttpTransportOptions;
 import com.google.cloud.storage.Storage;
 import com.google.cloud.storage.StorageOptions;
 import com.google.common.collect.ImmutableList;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processors.gcp.AbstractGCPProcessor;
-import org.apache.nifi.util.StringUtils;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.proxy.ProxyConfiguration;
 
-import java.net.Authenticator;
-import java.net.InetSocketAddress;
-import java.net.PasswordAuthentication;
 import java.net.Proxy;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -72,15 +71,24 @@ public abstract class AbstractGCSProcessor extends AbstractGCPProcessor<Storage,
     }
 
     @Override
+    protected final Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+        ProxyConfiguration.validateProxySpec(validationContext, results, ProxyAwareTransportFactory.PROXY_SPECS);
+        customValidate(validationContext, results);
+        return results;
+    }
+
+    /**
+     * If sub-classes needs to implement any custom validation, override this method then add validation result to the results.
+     */
+    protected void customValidate(ValidationContext validationContext, Collection<ValidationResult> results) {
+    }
+
+    @Override
     protected StorageOptions getServiceOptions(ProcessContext context, GoogleCredentials credentials) {
         final String projectId = context.getProperty(PROJECT_ID).getValue();
         final Integer retryCount = context.getProperty(RETRY_COUNT).asInteger();
 
-        final String proxyHost = context.getProperty(PROXY_HOST).getValue();
-        final Integer proxyPort = context.getProperty(PROXY_PORT).asInteger();
-        final String proxyUser = context.getProperty(HTTP_PROXY_USERNAME).evaluateAttributeExpressions().getValue();
-        final String proxyPassword = context.getProperty(HTTP_PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
-
         StorageOptions.Builder storageOptionsBuilder = StorageOptions.newBuilder()
                 .setCredentials(credentials)
                 .setProjectId(projectId)
@@ -88,25 +96,26 @@ public abstract class AbstractGCSProcessor extends AbstractGCPProcessor<Storage,
                         .setMaxAttempts(retryCount)
                         .build());
 
-        if (!StringUtils.isBlank(proxyHost) && proxyPort > 0) {
-            storageOptionsBuilder.setTransportOptions(HttpTransportOptions.newBuilder().setHttpTransportFactory(new HttpTransportFactory() {
-                @Override
-                public HttpTransport create() {
-                    if (!StringUtils.isBlank(proxyUser) && !StringUtils.isBlank(proxyPassword)) {
-                        Authenticator authenticator = new Authenticator() {
-                            public PasswordAuthentication getPasswordAuthentication() {
-                                return (new PasswordAuthentication(proxyUser,
-                                        proxyPassword.toCharArray()));
-                            }
-                        };
-                        Authenticator.setDefault(authenticator);
-                    }
-                    return new NetHttpTransport.Builder()
-                            .setProxy(new Proxy(Proxy.Type.HTTP, new InetSocketAddress(proxyHost, proxyPort)))
-                            .build();
-                }
-            }).build());
-        }
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context, () -> {
+            final String proxyHost = context.getProperty(PROXY_HOST).getValue();
+            final Integer proxyPort = context.getProperty(PROXY_PORT).asInteger();
+            if (proxyHost != null && proxyPort != null && proxyPort > 0) {
+                final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
+                final String proxyUser = context.getProperty(HTTP_PROXY_USERNAME).evaluateAttributeExpressions().getValue();
+                final String proxyPassword = context.getProperty(HTTP_PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
+                componentProxyConfig.setProxyType(Proxy.Type.HTTP);
+                componentProxyConfig.setProxyServerHost(proxyHost);
+                componentProxyConfig.setProxyServerPort(proxyPort);
+                componentProxyConfig.setProxyUserName(proxyUser);
+                componentProxyConfig.setProxyUserPassword(proxyPassword);
+                return componentProxyConfig;
+            }
+            return ProxyConfiguration.DIRECT_CONFIGURATION;
+        });
+
+        final ProxyAwareTransportFactory transportFactory = new ProxyAwareTransportFactory(proxyConfiguration);
+        storageOptionsBuilder.setTransportOptions(HttpTransportOptions.newBuilder().setHttpTransportFactory(transportFactory).build());
+
         return  storageOptionsBuilder.build();
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.java
index 290ec83..6ee8c10 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.java
@@ -16,6 +16,9 @@
  */
 package org.apache.nifi.processors.gcp.credentials.factory;
 
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.auth.http.HttpTransportFactory;
 import com.google.auth.oauth2.ComputeEngineCredentials;
 import com.google.auth.oauth2.GoogleCredentials;
 import com.google.auth.oauth2.ServiceAccountCredentials;
@@ -39,6 +42,9 @@ import static org.junit.Assert.assertTrue;
  */
 public class CredentialsFactoryTest {
 
+    private static final HttpTransport TRANSPORT = new NetHttpTransport();
+    private static final HttpTransportFactory TRANSPORT_FACTORY = () -> TRANSPORT;
+
     @Test
     public void testCredentialPropertyDescriptorClassCannotBeInvoked() throws Exception {
         Constructor constructor = CredentialPropertyDescriptors.class.getDeclaredConstructor();
@@ -54,7 +60,7 @@ public class CredentialsFactoryTest {
 
         Map<PropertyDescriptor, String> properties = runner.getProcessContext().getProperties();
         final CredentialsFactory factory = new CredentialsFactory();
-        final GoogleCredentials credentials = factory.getGoogleCredentials(properties);
+        final GoogleCredentials credentials = factory.getGoogleCredentials(properties, TRANSPORT_FACTORY);
 
         assertNotNull(credentials);
     }
@@ -67,7 +73,7 @@ public class CredentialsFactoryTest {
 
         Map<PropertyDescriptor, String> properties = runner.getProcessContext().getProperties();
         final CredentialsFactory factory = new CredentialsFactory();
-        final GoogleCredentials credentials = factory.getGoogleCredentials(properties);
+        final GoogleCredentials credentials = factory.getGoogleCredentials(properties, TRANSPORT_FACTORY);
 
         assertNotNull(credentials);
     }
@@ -89,7 +95,7 @@ public class CredentialsFactoryTest {
 
         Map<PropertyDescriptor, String> properties = runner.getProcessContext().getProperties();
         final CredentialsFactory factory = new CredentialsFactory();
-        final GoogleCredentials credentials = factory.getGoogleCredentials(properties);
+        final GoogleCredentials credentials = factory.getGoogleCredentials(properties, TRANSPORT_FACTORY);
 
         assertNotNull(credentials);
         assertEquals("credentials class should be equal", ServiceAccountCredentials.class,
@@ -117,7 +123,7 @@ public class CredentialsFactoryTest {
 
         Map<PropertyDescriptor, String> properties = runner.getProcessContext().getProperties();
         final CredentialsFactory factory = new CredentialsFactory();
-        final GoogleCredentials credentials = factory.getGoogleCredentials(properties);
+        final GoogleCredentials credentials = factory.getGoogleCredentials(properties, TRANSPORT_FACTORY);
 
         assertNotNull(credentials);
         assertEquals("credentials class should be equal", ServiceAccountCredentials.class,
@@ -132,7 +138,7 @@ public class CredentialsFactoryTest {
 
         Map<PropertyDescriptor, String> properties = runner.getProcessContext().getProperties();
         final CredentialsFactory factory = new CredentialsFactory();
-        final GoogleCredentials credentials = factory.getGoogleCredentials(properties);
+        final GoogleCredentials credentials = factory.getGoogleCredentials(properties, TRANSPORT_FACTORY);
 
         assertNotNull(credentials);
         assertEquals("credentials class should be equal", ComputeEngineCredentials.class,

http://git-wip-us.apache.org/repos/asf/nifi/blob/5a58c9a1/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml
index ed74d5b..20f1b70 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api-nar/pom.xml
@@ -33,6 +33,12 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>1.8.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-gcp-services-api</artifactId>
             <version>1.8.0-SNAPSHOT</version>
         </dependency>