You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ex...@apache.org on 2021/08/31 00:20:12 UTC

[nifi] branch main updated: NIFI-6616 Added GCP Sensitive Property Provider

This is an automated email from the ASF dual-hosted git repository.

exceptionfactory pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new 2ab23ef  NIFI-6616 Added GCP Sensitive Property Provider
2ab23ef is described below

commit 2ab23efb743e004f67d30431a3f264d1a54f3611
Author: Emilio Setiadarma <em...@gmail.com>
AuthorDate: Thu Jul 22 11:36:14 2021 -0700

    NIFI-6616 Added GCP Sensitive Property Provider
    
    This closes #5242
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 nifi-assembly/NOTICE                               |   2 +-
 .../nifi/properties/BootstrapProperties.java       |   1 +
 .../nifi-sensitive-property-provider/pom.xml       |  22 ++
 .../AWSKMSSensitivePropertyProvider.java           |   4 +-
 .../GCPKMSSensitivePropertyProvider.java           | 288 +++++++++++++++++++++
 .../nifi/properties/PropertyProtectionScheme.java  |   1 +
 .../StandardSensitivePropertyProviderFactory.java  |   2 +
 .../AWSKMSSensitivePropertyProviderIT.java         |  11 +-
 ...java => GCPKMSSensitivePropertyProviderIT.java} |  77 +++---
 .../src/main/asciidoc/administration-guide.adoc    |  15 +-
 nifi-docs/src/main/asciidoc/toolkit-guide.adoc     |  37 +--
 .../src/main/resources/conf/bootstrap-gcp.conf     |  22 ++
 .../src/main/resources/conf/bootstrap.conf         |   3 +
 .../nifi-grpc-bundle/nifi-grpc-processors/pom.xml  |   7 -
 .../src/main/resources/conf/bootstrap-gcp.conf     |  22 ++
 .../src/main/resources/conf/bootstrap.conf         |   5 +-
 16 files changed, 443 insertions(+), 76 deletions(-)

diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE
index 55ebeda..a47342d 100644
--- a/nifi-assembly/NOTICE
+++ b/nifi-assembly/NOTICE
@@ -921,7 +921,7 @@ The following binary components are provided under the Apache Software License v
   (ASLv2) Google Cloud Client Library for Java
     The following NOTICE information applies:
       Google Cloud Client Library for Java
-      Copyright 2016 Google Inc. All Rights Reserved.
+      Copyright 2016-2021 Google Inc. All Rights Reserved.
 
   (ASLv2) Guava
     The following NOTICE information applies:
diff --git a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
index d38b747..cc31acf 100644
--- a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
+++ b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
@@ -36,6 +36,7 @@ public class BootstrapProperties extends StandardReadableProperties {
         HASHICORP_VAULT_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.hashicorp.vault.conf"),
         AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.aws.kms.conf"),
         AZURE_KEYVAULT_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.azure.keyvault.conf"),
+        GCP_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.gcp.kms.conf"),
         CONTEXT_MAPPING_PREFIX("bootstrap.protection.context.mapping.");
 
         private final String key;
diff --git a/nifi-commons/nifi-sensitive-property-provider/pom.xml b/nifi-commons/nifi-sensitive-property-provider/pom.xml
index d680559..5ef4a43 100644
--- a/nifi-commons/nifi-sensitive-property-provider/pom.xml
+++ b/nifi-commons/nifi-sensitive-property-provider/pom.xml
@@ -23,7 +23,19 @@
     <artifactId>nifi-sensitive-property-provider</artifactId>
     <properties>
         <aws.sdk.version>2.17.1</aws.sdk.version>
+        <gcp.sdk.version>22.0.0</gcp.sdk.version>
     </properties>
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>com.google.cloud</groupId>
+                <artifactId>libraries-bom</artifactId>
+                <version>${gcp.sdk.version}</version>
+                <type>pom</type>
+                <scope>import</scope>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -112,6 +124,16 @@
             <version>${aws.sdk.version}</version>
         </dependency>
         <dependency>
+            <groupId>com.google.cloud</groupId>
+            <artifactId>google-cloud-kms</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>commons-logging</groupId>
+                    <artifactId>commons-logging</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-vault-utils</artifactId>
             <version>1.15.0-SNAPSHOT</version>
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProvider.java
index 3386488..35a04af 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProvider.java
@@ -277,7 +277,7 @@ public class AWSKMSSensitivePropertyProvider extends AbstractSensitivePropertyPr
 
     /**
      * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
-     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     * Encrypts a sensitive value using a key managed by AWS Key Management Service.
      *
      * @param unprotectedValue the sensitive value.
      * @param context The context of the value (ignored in this implementation)
@@ -302,7 +302,7 @@ public class AWSKMSSensitivePropertyProvider extends AbstractSensitivePropertyPr
 
     /**
      * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
-     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     * Decrypts a secured value from a ciphertext using a key managed by AWS Key Management Service.
      *
      * @param protectedValue the protected value read from the {@code nifi.properties} file.
      * @param context The context of the value (ignored in this implementation)
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/GCPKMSSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/GCPKMSSensitivePropertyProvider.java
new file mode 100644
index 0000000..0322230
--- /dev/null
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/GCPKMSSensitivePropertyProvider.java
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import com.google.api.gax.rpc.ApiException;
+import com.google.cloud.kms.v1.CryptoKey;
+import com.google.cloud.kms.v1.CryptoKeyName;
+import com.google.cloud.kms.v1.CryptoKeyVersion;
+import com.google.cloud.kms.v1.DecryptResponse;
+import com.google.cloud.kms.v1.EncryptResponse;
+import com.google.cloud.kms.v1.KeyManagementServiceClient;
+import com.google.protobuf.ByteString;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Base64;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class GCPKMSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(GCPKMSSensitivePropertyProvider.class);
+
+    private static final String GCP_PREFIX = "gcp";
+    private static final String PROJECT_ID_PROPS_NAME = "gcp.kms.project";
+    private static final String LOCATION_ID_PROPS_NAME = "gcp.kms.location";
+    private static final String KEYRING_ID_PROPS_NAME = "gcp.kms.keyring";
+    private static final String KEY_ID_PROPS_NAME = "gcp.kms.key";
+
+    private static final Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties gcpBootstrapProperties;
+    private KeyManagementServiceClient client;
+    private CryptoKeyName keyName;
+
+    GCPKMSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to GCP SPP is null");
+        gcpBootstrapProperties = getGCPBootstrapProperties(bootstrapProperties);
+        loadRequiredGCPProperties(gcpBootstrapProperties);
+    }
+
+    /**
+     * Initializes the GCP KMS Client to be used for encrypt, decrypt and other interactions with GCP Cloud KMS.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        try {
+            client = KeyManagementServiceClient.create();
+        } catch (final IOException e) {
+            final String msg = "Encountered an error initializing GCP Cloud KMS client";
+            throw new SensitivePropertyProtectionException(msg, e);
+        }
+    }
+
+    /**
+     * Validates the key details provided by the user.
+     */
+    private void validate() throws ApiException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The GCP KMS client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (keyName == null) {
+            final String msg = "The GCP KMS key provided is not provided/complete";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        final CryptoKey key;
+        final CryptoKeyVersion keyVersion;
+        try {
+            key = client.getCryptoKey(keyName);
+            keyVersion = client.getCryptoKeyVersion(key.getPrimary().getName());
+        } catch (final ApiException e) {
+            throw new SensitivePropertyProtectionException("Encountered an error while fetching key details", e);
+        }
+
+        if (keyVersion.getState() != CryptoKeyVersion.CryptoKeyVersionState.ENABLED) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");
+        }
+    }
+
+    /**
+     * Checks if we have the required key properties for GCP Cloud KMS and loads it into {@link #keyName}.
+     * Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-gcp.conf.
+     */
+    private void loadRequiredGCPProperties(final BootstrapProperties props) {
+        if (props != null) {
+            final String projectId = props.getProperty(PROJECT_ID_PROPS_NAME);
+            final String locationId = props.getProperty(LOCATION_ID_PROPS_NAME);
+            final String keyRingId = props.getProperty(KEYRING_ID_PROPS_NAME);
+            final String keyId = props.getProperty(KEY_ID_PROPS_NAME);
+            if (StringUtils.isNoneBlank(projectId, locationId, keyRingId, keyId)) {
+                keyName = CryptoKeyName.of(projectId, locationId, keyRingId, keyId);
+            }
+        }
+    }
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.GCP_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-gcp.conf file. Also will load bootstrap-gcp.conf to {@link #gcpBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-gcp.conf, null otherwise.
+     */
+    private BootstrapProperties getGCPBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-gcp.conf file based on path specified in
+        // "nifi.bootstrap.protection.gcp.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.GCP_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("GCP KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), GCP_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-gcp.conf for the required configurations for Google Cloud KMS encrypt/decrypt operations.
+     * @return true if bootstrap-gcp.conf contains the required properties for GCP KMS SPP, false otherwise.
+     */
+    private boolean hasRequiredGCPProperties() {
+        if (gcpBootstrapProperties == null) {
+            return false;
+        }
+
+        final String projectId = gcpBootstrapProperties.getProperty(PROJECT_ID_PROPS_NAME);
+        final String locationId = gcpBootstrapProperties.getProperty(LOCATION_ID_PROPS_NAME);
+        final String keyRingId = gcpBootstrapProperties.getProperty(KEYRING_ID_PROPS_NAME);
+        final String keyId = gcpBootstrapProperties.getProperty(KEY_ID_PROPS_NAME);
+
+        // Note: the following does not verify if the properties are valid properties, they only verify if
+        // the properties are configured in bootstrap-gcp.conf.
+        return StringUtils.isNoneBlank(projectId, locationId, keyRingId, keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredGCPProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.GCP_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.GCP_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.GCP_KMS.getIdentifier();
+    }
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using a key stored in GCP KMS.
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) throws IOException {
+        final EncryptResponse response = client.encrypt(keyName, ByteString.copyFrom(input));
+        return response.getCiphertext().toByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using a key stored in GCP KMS.
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) throws IOException {
+        final DecryptResponse response = client.decrypt(keyName, ByteString.copyFrom(input));
+        return response.getPlaintext().toByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for GCP KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the GCP KMS client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * Encrypts a sensitive value using a key managed by Google Cloud Key Management Service.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @param context The context of the value (ignored in this implementation)
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.getEncoder().encodeToString(cipherBytes);
+        } catch (final IOException | ApiException e) {
+            throw new SensitivePropertyProtectionException("Encrypt failed", e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * Decrypts a secured value from a ciphertext using a key managed by Google Cloud Key Management Service.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @param context The context of the value (ignored in this implementation)
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            byte[] cipherBytes = Base64.getDecoder().decode(protectedValue);
+            byte[] plainBytes = decrypt(cipherBytes);
+            return new String(plainBytes, PROPERTY_CHARSET);
+        } catch (final IOException | ApiException e) {
+            throw new SensitivePropertyProtectionException("Decrypt failed", e);
+        }
+    }
+
+    /**
+     * Closes GCP KMS client that may have been opened.
+     */
+    @Override
+    public void cleanUp() {
+        if (client != null) {
+            client.close();
+            client = null;
+        }
+    }
+}
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
index 1eb565b..157fc81 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
@@ -27,6 +27,7 @@ public enum PropertyProtectionScheme {
     AES_GCM("aes/gcm/(128|192|256)", "aes/gcm/%s", "AES Sensitive Property Provider", true),
     AWS_KMS("aws/kms", "aws/kms", "AWS KMS Sensitive Property Provider", false),
     AZURE_KEYVAULT_KEY("azure/keyvault/key", "azure/keyvault/key", "Azure Key Vault Key Sensitive Property Provider", false),
+    GCP_KMS("gcp/kms", "gcp/kms", "GCP Cloud KMS Sensitive Property Provider", false),
     HASHICORP_VAULT_KV("hashicorp/vault/kv/[a-zA-Z0-9_-]+", "hashicorp/vault/kv/%s", "HashiCorp Vault Key/Value Engine Sensitive Property Provider", false),
     HASHICORP_VAULT_TRANSIT("hashicorp/vault/transit/[a-zA-Z0-9_-]+", "hashicorp/vault/transit/%s", "HashiCorp Vault Transit Engine Sensitive Property Provider", false);
 
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
index 4439d74..a64bd0a 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
@@ -124,6 +124,8 @@ public class StandardSensitivePropertyProviderFactory implements SensitiveProper
                 return providerMap.computeIfAbsent(protectionScheme, s -> new AWSKMSSensitivePropertyProvider(getBootstrapProperties()));
             case AZURE_KEYVAULT_KEY:
                 return providerMap.computeIfAbsent(protectionScheme, s -> new AzureKeyVaultKeySensitivePropertyProvider(getBootstrapProperties()));
+            case GCP_KMS:
+                return providerMap.computeIfAbsent(protectionScheme, s -> new GCPKMSSensitivePropertyProvider(getBootstrapProperties()));
             case HASHICORP_VAULT_TRANSIT:
                 return providerMap.computeIfAbsent(protectionScheme, s -> new HashiCorpVaultTransitSensitivePropertyProvider(getBootstrapProperties()));
             case HASHICORP_VAULT_KV:
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
index a22945c..d29126d 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
@@ -88,10 +88,11 @@ public class AWSKMSSensitivePropertyProviderIT {
         String keyId = System.getProperty(KMS_KEY_PROPS_NAME, EMPTY_PROPERTY);
 
         StringBuilder bootstrapConfText = new StringBuilder();
+        String lineSeparator = System.getProperty("line.separator");
         bootstrapConfText.append(ACCESS_KEY_PROPS_NAME + "=" + accessKey);
-        bootstrapConfText.append("\n" + SECRET_KEY_PROPS_NAME + "=" + secretKey);
-        bootstrapConfText.append("\n" + REGION_KEY_PROPS_NAME + "=" + region);
-        bootstrapConfText.append("\n" + KMS_KEY_PROPS_NAME + "=" + keyId);
+        bootstrapConfText.append(lineSeparator + SECRET_KEY_PROPS_NAME + "=" + secretKey);
+        bootstrapConfText.append(lineSeparator + REGION_KEY_PROPS_NAME + "=" + region);
+        bootstrapConfText.append(lineSeparator + KMS_KEY_PROPS_NAME + "=" + keyId);
 
         IOUtil.writeText(bootstrapConfText.toString(), mockAWSBootstrapConf.toFile());
     }
@@ -114,9 +115,9 @@ public class AWSKMSSensitivePropertyProviderIT {
 
     @Test
     public void testEncryptDecrypt() {
-        logger.info("Running testEncryptDecrypt of AWS SPP integration test");
+        logger.info("Running testEncryptDecrypt of AWS KMS SPP integration test");
         runEncryptDecryptTest();
-        logger.info("testEncryptDecrypt of AWS SPP integration test completed");
+        logger.info("testEncryptDecrypt of AWS KMS SPP integration test completed");
     }
 
     private static void runEncryptDecryptTest() {
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/GCPKMSSensitivePropertyProviderIT.java
similarity index 54%
copy from nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
copy to nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/GCPKMSSensitivePropertyProviderIT.java
index a22945c..64c58da 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSKMSSensitivePropertyProviderIT.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/GCPKMSSensitivePropertyProviderIT.java
@@ -32,49 +32,42 @@ import java.util.Properties;
 
 /**
  * To run this test, make sure to first configure sensitive credential information as in the following link
- * https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-files.html
+ * https://cloud.google.com/kms/docs/reference/libraries#cloud-console
  *
- * If you don't have a key then run:
- * aws kms create-key
+ * Create a project, keyring and key in the web console.
  *
- * Take note of the key id or arn.
- *
- * Then, set the system property -Daws.kms.key.id to the either key id value or arn value
- *
- * The following settings are optional. If you have a default AWS configuration and credentials in ~/.aws then
- * it will take that. Otherwise you can set all of the following:
- * set the system property -Daws.access.key.id to the access key id
- * set the system property -Daws.secret.access.key to the secret access key
- * set the system property -Daws.region to the region
- *
- * After you are satisfied with the test, and you don't need the key, you may schedule key deletion with:
- * aws kms schedule-key-deletion --key-id "key id" --pending-window-in-days "number of days"
+ * Take note of the project name, location, keyring name and key name.
  *
+ * Then, set the system properties as follows:
+ * -Dgcp.kms.project="project"
+ * -Dgcp.kms.location="location"
+ * -Dgcp.kms.keyring="key ring name"
+ * -Dgcp.kms.key="key name"
+ * when running the integration tests
  */
 
-public class AWSKMSSensitivePropertyProviderIT {
-    private static final String SAMPLE_PLAINTEXT = "AWSKMSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
-    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
-    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.access.key";
-    private static final String REGION_KEY_PROPS_NAME = "aws.region";
-    private static final String KMS_KEY_PROPS_NAME = "aws.kms.key.id";
-
-    private static final String BOOTSTRAP_AWS_FILE_PROPS_NAME = "nifi.bootstrap.protection.aws.kms.conf";
+public class GCPKMSSensitivePropertyProviderIT {
+    private static final String SAMPLE_PLAINTEXT = "GCPKMSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
+    private static final String PROJECT_ID_PROPS_NAME = "gcp.kms.project";
+    private static final String LOCATION_ID_PROPS_NAME = "gcp.kms.location";
+    private static final String KEYRING_ID_PROPS_NAME = "gcp.kms.keyring";
+    private static final String KEY_ID_PROPS_NAME = "gcp.kms.key";
+    private static final String BOOTSTRAP_GCP_FILE_PROPS_NAME = "nifi.bootstrap.protection.gcp.kms.conf";
 
     private static final String EMPTY_PROPERTY = "";
 
-    private static AWSKMSSensitivePropertyProvider spp;
+    private static GCPKMSSensitivePropertyProvider spp;
 
     private static BootstrapProperties props;
 
-    private static Path mockBootstrapConf, mockAWSBootstrapConf;
+    private static Path mockBootstrapConf, mockGCPBootstrapConf;
 
-    private static final Logger logger = LoggerFactory.getLogger(AWSKMSSensitivePropertyProviderIT.class);
+    private static final Logger logger = LoggerFactory.getLogger(GCPKMSSensitivePropertyProviderIT.class);
 
     private static void initializeBootstrapProperties() throws IOException{
         mockBootstrapConf = Files.createTempFile("bootstrap", ".conf").toAbsolutePath();
-        mockAWSBootstrapConf = Files.createTempFile("bootstrap-aws", ".conf").toAbsolutePath();
-        IOUtil.writeText(BOOTSTRAP_AWS_FILE_PROPS_NAME + "=" + mockAWSBootstrapConf.toAbsolutePath(), mockBootstrapConf.toFile());
+        mockGCPBootstrapConf = Files.createTempFile("bootstrap-gcp", ".conf").toAbsolutePath();
+        IOUtil.writeText(BOOTSTRAP_GCP_FILE_PROPS_NAME + "=" + mockGCPBootstrapConf.toAbsolutePath(), mockBootstrapConf.toFile());
 
         final Properties bootstrapProperties = new Properties();
         try (final InputStream inputStream = Files.newInputStream(mockBootstrapConf)) {
@@ -82,41 +75,41 @@ public class AWSKMSSensitivePropertyProviderIT {
             props = new BootstrapProperties("nifi", bootstrapProperties, mockBootstrapConf);
         }
 
-        String accessKey = System.getProperty(ACCESS_KEY_PROPS_NAME, EMPTY_PROPERTY);
-        String secretKey = System.getProperty(SECRET_KEY_PROPS_NAME, EMPTY_PROPERTY);
-        String region = System.getProperty(REGION_KEY_PROPS_NAME, EMPTY_PROPERTY);
-        String keyId = System.getProperty(KMS_KEY_PROPS_NAME, EMPTY_PROPERTY);
+        String projectId = System.getProperty(PROJECT_ID_PROPS_NAME, EMPTY_PROPERTY);
+        String locationId = System.getProperty(LOCATION_ID_PROPS_NAME, EMPTY_PROPERTY);
+        String keyringId = System.getProperty(KEYRING_ID_PROPS_NAME, EMPTY_PROPERTY);
+        String keyId = System.getProperty(KEY_ID_PROPS_NAME, EMPTY_PROPERTY);
 
         StringBuilder bootstrapConfText = new StringBuilder();
-        bootstrapConfText.append(ACCESS_KEY_PROPS_NAME + "=" + accessKey);
-        bootstrapConfText.append("\n" + SECRET_KEY_PROPS_NAME + "=" + secretKey);
-        bootstrapConfText.append("\n" + REGION_KEY_PROPS_NAME + "=" + region);
-        bootstrapConfText.append("\n" + KMS_KEY_PROPS_NAME + "=" + keyId);
-
-        IOUtil.writeText(bootstrapConfText.toString(), mockAWSBootstrapConf.toFile());
+        String lineSeparator = System.getProperty("line.separator");
+        bootstrapConfText.append(PROJECT_ID_PROPS_NAME + "=" + projectId);
+        bootstrapConfText.append(lineSeparator + LOCATION_ID_PROPS_NAME + "=" + locationId);
+        bootstrapConfText.append(lineSeparator + KEYRING_ID_PROPS_NAME + "=" + keyringId);
+        bootstrapConfText.append(lineSeparator + KEY_ID_PROPS_NAME + "=" + keyId);
+        IOUtil.writeText(bootstrapConfText.toString(), mockGCPBootstrapConf.toFile());
     }
 
     @BeforeClass
     public static void initOnce() throws IOException {
         initializeBootstrapProperties();
         Assert.assertNotNull(props);
-        spp = new AWSKMSSensitivePropertyProvider(props);
+        spp = new GCPKMSSensitivePropertyProvider(props);
         Assert.assertNotNull(spp);
     }
 
     @AfterClass
     public static void tearDownOnce() throws IOException {
         Files.deleteIfExists(mockBootstrapConf);
-        Files.deleteIfExists(mockAWSBootstrapConf);
+        Files.deleteIfExists(mockGCPBootstrapConf);
 
         spp.cleanUp();
     }
 
     @Test
     public void testEncryptDecrypt() {
-        logger.info("Running testEncryptDecrypt of AWS SPP integration test");
+        logger.info("Running testEncryptDecrypt of GCP KMS SPP integration test");
         runEncryptDecryptTest();
-        logger.info("testEncryptDecrypt of AWS SPP integration test completed");
+        logger.info("testEncryptDecrypt of GCP KMS SPP integration test completed");
     }
 
     private static void runEncryptDecryptTest() {
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 46f25ae..1120695 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -1869,6 +1869,19 @@ This protection scheme uses keys managed by https://docs.microsoft.com/en-us/azu
 |`azure.keyvault.encryption.algorithm`|The encryption algorithm that the Azure Key Vault client uses for encryption and decryption.|_none_
 |===
 
+=== Google Cloud KMS provider
+This protection scheme uses Google Cloud Key Management Service (https://cloud.google.com/security-key-management[Google Cloud Key Management Service]) for encryption and decryption. Google Cloud KMS configuration properties are to be stored in the `bootstrap-gcp.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. Credentials must be configured as per the following documentation: https://cloud.google.com/kms/docs/reference/libraries[Google Cloud KMS documentation]
+
+==== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`gcp.kms.project`|The project containing the key that the Google Cloud KMS client uses for encryption and decryption.|_none_
+|`gcp.kms.location`|The geographic region of the project containing the key that the Google Cloud KMS client uses for encryption and decryption.|_none_
+|`gcp.kms.keyring`|The keyring containing the key that the Google Cloud KMS client uses for encryption and decryption.|_none_
+|`gcp.kms.key`|The key identifier that the Google Cloud KMS client uses for encryption and decryption.|_none_
+|===
+
 === Property Context Mapping
 Some encryption providers store protected values in an external service instead of persisting the encrypted values directly in the configuration file.  To support this use case, a property context is defined for each protected property in NiFi's configuration files, in the format: `{context-name}/{property-name}`
 
@@ -4313,4 +4326,4 @@ The verbose switch is optional and can be used to control the level of diagnosti
 === Automatic diagnostics on restart and shutdown
 
 NiFi supports automatic diagnostics in the event of a shutdown. The feature is disabled by default. The settings can be found in the nifi.properties file and the feature can be enabled there also.
-In the case of a lengthy diagnostic, NiFi may terminate before the diagnostics are completed. In this case, the graceful.shutdown.seconds property should be set to a higher value in the bootstrap.conf.
\ No newline at end of file
+In the case of a lengthy diagnostic, NiFi may terminate before the diagnostics are completed. In this case, the graceful.shutdown.seconds property should be set to a higher value in the bootstrap.conf.
diff --git a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
index 1738fa3..968def6 100644
--- a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
@@ -435,7 +435,7 @@ The following are available options when targeting NiFi:
 * `-f`,`--flowXml <file>`                       The _flow.xml.gz_ file currently protected with old password (will be overwritten unless `-g` is specified)
 * `-g`,`--outputFlowXml <file>`                 The destination _flow.xml.gz_ file containing protected config values (will not modify input _flow.xml.gz_)
 * `-b`,`--bootstrapConf <file>`                 The bootstrap.conf file to persist root key and to optionally provide any configuration for the protection scheme.
-* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>] (default is AES_GCM)
+* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>, <<GCP_KMS>>] (default is AES_GCM)
 * `-k`,`--key <keyhex>`                         The raw hexadecimal key to use to encrypt the sensitive properties
 * `-e`,`--oldKey <keyhex>`                      The old raw hexadecimal key to use during key migration
 * `-H`,`--oldProtectionScheme <protectionScheme>` The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM
@@ -456,7 +456,7 @@ The following are available options when targeting NiFi Registry using the `--ni
 * `-v`,`--verbose`                              Sets verbose mode (default false)
 * `-p`,`--password <password>`                  Protect the files using a password-derived key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the password.
 * `-k`,`--key <keyhex>`                         Protect the files using a raw hexadecimal key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the key.
-* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>]  (default is AES_GCM)
+* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>, <<GCP_KMS>>]  (default is AES_GCM)
 * `--oldPassword <password>`                    If the input files are already protected using a password-derived key, this specifies the old password so that the files can be unprotected before re-protecting.
 * `--oldKey <keyhex>`                           If the input files are already protected using a key, this specifies the raw hexadecimal key so that the files can be unprotected before re-protecting.
 * `-H`,`--oldProtectionScheme <protectionScheme>`The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM.
@@ -488,6 +488,9 @@ This protection scheme uses https://aws.amazon.com/kms/[AWS Key Management] Serv
 ==== AZURE_KEYVAULT_KEY [[AZURE_KEYVAULT_KEY]]
 This protection scheme uses keys managed by https://docs.microsoft.com/en-us/azure/key-vault/keys/about-keys[Azure Key Vault Keys] for encryption and decryption. Azure Key Vault configuration properties can be stored in the `bootstrap-azure.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. The provider will utilize the Azure default credentials provider chain as described in the https://docs.microsoft.com/en-us/java/api/overview/azure/security-keyvault-keys-read [...]
 
+==== GCP_KMS [[GCP_KMS]]
+This protection scheme uses Google Cloud Key Management Service (https://cloud.google.com/security-key-management[Google Cloud Key Management Service]) for encryption and decryption. Google Cloud KMS configuration properties are to be stored in the `bootstrap-gcp.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. Credentials must be configured as per the following documentation: https://cloud.google.com/kms/docs/reference/libraries[Google Cloud KMS documentation] [...]
+
 === Examples
 
 ==== NiFi
@@ -1120,21 +1123,21 @@ In some enterprise scenarios, a security/IT team may provide a signing certifica
 🔓 0s @ 18:07:58 $ tree -L 2
 .
 ├── hardcoded
-│   ├── CN=myusername.hardcoded_OU=NiFi.p12
-│   ├── CN=myusername.hardcoded_OU=NiFi.password
-│   ├── nifi-cert.pem
-│   ├── nifi-key.key
-│   ├── node1.nifi.apache.org
-│   ├── node2.nifi.apache.org
-│   └── node3.nifi.apache.org
+│   ├── CN=myusername.hardcoded_OU=NiFi.p12
+│   ├── CN=myusername.hardcoded_OU=NiFi.password
+│   ├── nifi-cert.pem
+│   ├── nifi-key.key
+│   ├── node1.nifi.apache.org
+│   ├── node2.nifi.apache.org
+│   └── node3.nifi.apache.org
 └── toolkit
-    ├── LICENSE
-    ├── NOTICE
-    ├── README
-    ├── bin
-    ├── conf
-    ├── docs
-    └── lib
+    ├── LICENSE
+    ├── NOTICE
+    ├── README
+    ├── bin
+    ├── conf
+    ├── docs
+    └── lib
 ----
 
 The `nifi-cert.pem` and `nifi-key.key` files should be ASCII-armored (Base64-encoded ASCII) files containing the CA public certificate and private key respectively. Here are sample files of each to show the expected format:
@@ -1527,4 +1530,4 @@ NOTE: As of NiFi 1.10.x, because of an upgrade to ZooKeeper 3.5.x, the migrator
 * For a ZooKeeper using Kerberos for authentication:
 ** `zk-migrator.sh -s -z destinationHostname:destinationClientPort/destinationRootPath/components -k /path/to/jaasconfig/jaas-config.conf -f /path/to/export/zk-source-data.json`
 
-6. Once the migration has completed successfully, start the processors in the NiFi flow.  Processing should continue from the point at which it was stopped when the NiFi flow was stopped.
+6. Once the migration has completed successfully, start the processors in the NiFi flow.  Processing should continue from the point at which it was stopped when the NiFi flow was stopped.
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-gcp.conf b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-gcp.conf
new file mode 100644
index 0000000..440dad2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-gcp.conf
@@ -0,0 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# These GCP KMS settings must all be configured in order to use the GCP KMS Sensitive Property Provider
+gcp.kms.project=
+gcp.kms.location=
+gcp.kms.keyring=
+gcp.kms.key=
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
index 4f348c2..5390153 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
@@ -69,6 +69,9 @@ nifi.bootstrap.protection.aws.kms.conf=./conf/bootstrap-aws.conf
 # Azure Key Vault Sensitive Property Providers
 nifi.bootstrap.protection.azure.keyvault.conf=./conf/bootstrap-azure.conf
 
+# GCP KMS Sensitive Property Providers
+nifi.bootstrap.protection.gcp.kms.conf=./conf/bootstrap-gcp.conf
+
 # Sets the provider of SecureRandom to /dev/urandom to prevent blocking on VMs
 java.arg.15=-Djava.security.egd=file:/dev/urandom
 
diff --git a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
index e527835..ef0827a 100644
--- a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
@@ -75,13 +75,6 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-mock</artifactId>
             <version>1.15.0-SNAPSHOT</version>
             <scope>test</scope>
-            <!-- Exclude transitive dependency of software.amazon.awssdk:kms included in nifi-sensitive-property-provider -->
-            <exclusions>
-                <exclusion>
-                    <artifactId>netty-transport-native-epoll</artifactId>
-                    <groupId>io.netty</groupId>
-                </exclusion>
-            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-gcp.conf b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-gcp.conf
new file mode 100644
index 0000000..440dad2
--- /dev/null
+++ b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-gcp.conf
@@ -0,0 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# These GCP KMS settings must all be configured in order to use the GCP KMS Sensitive Property Provider
+gcp.kms.project=
+gcp.kms.location=
+gcp.kms.keyring=
+gcp.kms.key=
\ No newline at end of file
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
index 2bcbc2e..ee03964 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
+++ b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
@@ -62,4 +62,7 @@ nifi.registry.bootstrap.protection.hashicorp.vault.conf=./conf/bootstrap-hashico
 nifi.registry.bootstrap.protection.aws.kms.conf=./conf/bootstrap-aws.conf
 
 # Azure Key Vault Sensitive Property Providers
-nifi.registry.bootstrap.protection.azure.keyvault.conf=./conf/bootstrap-azure.conf
\ No newline at end of file
+nifi.registry.bootstrap.protection.azure.keyvault.conf=./conf/bootstrap-azure.conf
+
+# GCP KMS Sensitive Property Providers
+nifi.registry.bootstrap.protection.gcp.kms.conf=./conf/bootstrap-gcp.conf