You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2020/12/09 14:29:59 UTC

[nifi] branch main updated: NIFI-7885 Added Environment Variable to deny LFS access using Hadoop (#4712)

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

bbende 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 0af736d  NIFI-7885 Added Environment Variable to deny LFS access using Hadoop (#4712)
0af736d is described below

commit 0af736dc0b1cb159555b42b61908266fc76bd96e
Author: exceptionfactory <ex...@gmail.com>
AuthorDate: Wed Dec 9 09:29:42 2020 -0500

    NIFI-7885 Added Environment Variable to deny LFS access using Hadoop (#4712)
    
    * NIFI-7885 Added Environment Variable to deny LFS access using Hadoop
    
    * NIFI-7885 Changed Hadoop Processor validation to check LFS access
---
 .../processors/hadoop/AbstractHadoopProcessor.java | 31 +++++++++++++++++++++
 .../src/main/resources/bin/nifi-env.sh             |  6 +++-
 .../nifi/processors/hadoop/AbstractHadoopTest.java | 32 ++++++++++++++++++++++
 .../processors/hadoop/SimpleHadoopProcessor.java   | 11 ++++++--
 4 files changed, 77 insertions(+), 3 deletions(-)

diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 7a59170..881237d 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@ -63,6 +63,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.WeakHashMap;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
 
 /**
  * This is a base class that is helpful when building processors interacting with HDFS.
@@ -79,6 +80,12 @@ import java.util.concurrent.atomic.AtomicReference;
 public abstract class AbstractHadoopProcessor extends AbstractProcessor {
     private static final String ALLOW_EXPLICIT_KEYTAB = "NIFI_ALLOW_EXPLICIT_KEYTAB";
 
+    private static final String DENY_LFS_ACCESS = "NIFI_HDFS_DENY_LOCAL_FILE_SYSTEM_ACCESS";
+
+    private static final String DENY_LFS_EXPLANATION = String.format("LFS Access Denied according to Environment Variable [%s]", DENY_LFS_ACCESS);
+
+    private static final Pattern LOCAL_FILE_SYSTEM_URI = Pattern.compile("^file:.*");
+
     // properties
     public static final PropertyDescriptor HADOOP_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
             .name("Hadoop Configuration Resources")
@@ -220,6 +227,14 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
             results.addAll(KerberosProperties.validatePrincipalWithKeytabOrPassword(
                 this.getClass().getSimpleName(), conf, resolvedPrincipal, resolvedKeytab, explicitPassword, getLogger()));
 
+            final URI fileSystemUri = FileSystem.getDefaultUri(conf);
+            if (isFileSystemAccessDenied(fileSystemUri)) {
+                results.add(new ValidationResult.Builder()
+                        .valid(false)
+                        .subject("Hadoop File System")
+                        .explanation(DENY_LFS_EXPLANATION)
+                        .build());
+            }
         } catch (final IOException e) {
             results.add(new ValidationResult.Builder()
                     .valid(false)
@@ -562,6 +577,22 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
         return Boolean.parseBoolean(System.getenv(ALLOW_EXPLICIT_KEYTAB));
     }
 
+    boolean isLocalFileSystemAccessDenied() {
+        return Boolean.parseBoolean(System.getenv(DENY_LFS_ACCESS));
+    }
+
+    private boolean isFileSystemAccessDenied(final URI fileSystemUri) {
+        boolean accessDenied;
+
+        if (isLocalFileSystemAccessDenied()) {
+            accessDenied = LOCAL_FILE_SYSTEM_URI.matcher(fileSystemUri.toString()).matches();
+        } else {
+            accessDenied = false;
+        }
+
+        return accessDenied;
+    }
+
     static protected class HdfsResources {
         private final Configuration configuration;
         private final FileSystem fileSystem;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi-env.sh b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi-env.sh
index c2b286a..5889d49 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi-env.sh
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/bin/nifi-env.sh
@@ -34,4 +34,8 @@ export NIFI_LOG_DIR="${NIFI_HOME}/logs"
 # a multi-tenant environment where management of keytabs should be performed only by
 # a user with elevated permissions (i.e., users that have been granted the 'ACCESS_KEYTAB'
 # restriction).
-export NIFI_ALLOW_EXPLICIT_KEYTAB=true
\ No newline at end of file
+export NIFI_ALLOW_EXPLICIT_KEYTAB=true
+
+# Set to true to deny access to the Local File System from HDFS Processors
+# This flag forces HDFS Processors to evaluate the File System path during scheduling
+export NIFI_HDFS_DENY_LOCAL_FILE_SYSTEM_ACCESS=false
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java
index 00a4851..cbd6559 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/AbstractHadoopTest.java
@@ -16,10 +16,12 @@
  */
 package org.apache.nifi.processors.hadoop;
 
+import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.hadoop.KerberosProperties;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.MockValidationContext;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -35,7 +37,10 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.Optional;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -173,4 +178,31 @@ public class AbstractHadoopTest {
         runner.assertNotValid();
 
     }
+
+    @Test
+    public void testLocalFileSystemInvalid() {
+        final SimpleHadoopProcessor processor = new SimpleHadoopProcessor(kerberosProperties, true, true);
+        TestRunner runner = TestRunners.newTestRunner(processor);
+        runner.setProperty(AbstractHadoopProcessor.HADOOP_CONFIGURATION_RESOURCES, "src/test/resources/core-site.xml");
+
+        final MockProcessContext processContext = (MockProcessContext) runner.getProcessContext();
+        final ValidationContext validationContext = new MockValidationContext(processContext);
+        final Collection<ValidationResult> results = processor.customValidate(validationContext);
+        final Optional<ValidationResult> optionalResult = results.stream()
+                .filter(result -> result.getSubject().equals("Hadoop File System"))
+                .findFirst();
+        assertTrue("Hadoop File System Validation Result not found", optionalResult.isPresent());
+        final ValidationResult result = optionalResult.get();
+        assertFalse("Hadoop File System Valid", result.isValid());
+    }
+
+    @Test
+    public void testDistributedFileSystemValid() {
+        final SimpleHadoopProcessor processor = new SimpleHadoopProcessor(kerberosProperties, true, true);
+        TestRunner runner = TestRunners.newTestRunner(processor);
+        runner.setProperty(AbstractHadoopProcessor.HADOOP_CONFIGURATION_RESOURCES, "src/test/resources/core-site-security.xml");
+        runner.setProperty(kerberosProperties.getKerberosPrincipal(), "principal");
+        runner.setProperty(kerberosProperties.getKerberosKeytab(), temporaryFile.getAbsolutePath());
+        runner.assertValid();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/SimpleHadoopProcessor.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/SimpleHadoopProcessor.java
index f8622da..0347b4f 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/SimpleHadoopProcessor.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/SimpleHadoopProcessor.java
@@ -27,14 +27,16 @@ public class SimpleHadoopProcessor extends AbstractHadoopProcessor {
 
     private KerberosProperties testKerberosProperties;
     private boolean allowExplicitKeytab;
+    private boolean localFileSystemAccessDenied;
 
     public SimpleHadoopProcessor(KerberosProperties kerberosProperties) {
-        this(kerberosProperties, true);
+        this(kerberosProperties, true, true);
     }
 
-    public SimpleHadoopProcessor(KerberosProperties kerberosProperties, boolean allowExplicitKeytab) {
+    public SimpleHadoopProcessor(KerberosProperties kerberosProperties, boolean allowExplicitKeytab, boolean localFileSystemAccessDenied) {
         this.testKerberosProperties = kerberosProperties;
         this.allowExplicitKeytab = allowExplicitKeytab;
+        this.localFileSystemAccessDenied = localFileSystemAccessDenied;
     }
 
     @Override
@@ -50,4 +52,9 @@ public class SimpleHadoopProcessor extends AbstractHadoopProcessor {
     boolean isAllowExplicitKeytab() {
         return allowExplicitKeytab;
     }
+
+    @Override
+    boolean isLocalFileSystemAccessDenied() {
+        return localFileSystemAccessDenied;
+    }
 }