markap14 commented on a change in pull request #5059:
URL: https://github.com/apache/nifi/pull/5059#discussion_r631964569



##########
File path: 
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/PropertyBasedNarProviderInitializationContext.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.nar;
+
+import org.apache.nifi.util.NiFiProperties;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A facade at front of {@code NiFiProperties} for auto loader extensions. 
Also limits the scope of the reachable properties.
+ */
+public class PropertyBasedNarProviderInitializationContext implements 
NarProviderInitializationContext {
+    private static Set<String> GUARDED_PROPERTIES = new 
HashSet<>(Arrays.asList("implementation"));
+    static final String BASIC_PREFIX = "nifi.nar.library.provider.";
+
+    private final Map<String, String> parameters;
+    private final String name;
+
+    public PropertyBasedNarProviderInitializationContext(final NiFiProperties 
properties, final String name) {
+        this.parameters = extractParameters(properties, name);
+        this.name = name;
+    }
+
+    @Override
+    public Map<String, String> getParameters() {

Review comment:
       I think we should call this `getProperties()` rather than 
`getParameters()` - within the NiFi parlance, both of these have very specific 
meanings. These really are configuration elements for the provider, coming from 
NiFiProperties, so I think `getProperties()` is more appropriate here.

##########
File path: 
nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/nar/hadoop/HDFSNarProvider.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nar.hadoop;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.nar.NarProvider;
+import org.apache.nifi.nar.NarProviderInitializationContext;
+import org.apache.nifi.nar.hadoop.util.ExtensionFilter;
+import org.apache.nifi.processors.hadoop.ExtendedConfiguration;
+import org.apache.nifi.processors.hadoop.HdfsResources;
+import org.apache.nifi.security.krb.KerberosKeytabUser;
+import org.apache.nifi.security.krb.KerberosPasswordUser;
+import org.apache.nifi.security.krb.KerberosUser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.SocketFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class HDFSNarProvider implements NarProvider {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(HDFSNarProvider.class);
+
+    private static final String RESOURCES_PARAMETER = "resources";
+    private static final String SOURCE_DIRECTORY_PARAMETER = 
"source.directory";
+    private static final String KERBEROS_PRINCIPAL_PARAMETER = 
"kerberos.principal";
+    private static final String KERBEROS_KEYTAB_PARAMETER = "kerberos.keytab";
+    private static final String KERBEROS_PASSWORD_PARAMETER = 
"kerberos.password";
+
+    private static final String NAR_EXTENSION = "nar";
+    private static final String DELIMITER = "/";
+    private static final int BUFFER_SIZE_DEFAULT = 4096;
+    private static final Object RESOURCES_LOCK = new Object();
+
+    private volatile List<String> resources = null;
+    private volatile Path sourceDirectory = null;
+
+    private volatile NarProviderInitializationContext context;
+
+    private volatile boolean initialized = false;
+
+    public void initialize(final NarProviderInitializationContext context) {
+        resources = 
Arrays.stream(Objects.requireNonNull(context.getParameters().get(RESOURCES_PARAMETER)).split(",")).map(s
 -> s.trim()).collect(Collectors.toList());
+
+        if (resources.isEmpty()) {
+            throw new IllegalArgumentException("At least one HDFS 
configuration resource is necessary");
+        }
+
+        this.sourceDirectory = new 
Path(Objects.requireNonNull(context.getParameters().get(SOURCE_DIRECTORY_PARAMETER)));

Review comment:
       While I believe this code is technically correct and will do what is 
intended when configured properly, I think it's going to result in a poor user 
experience if not configured properly. It also doesn't appear to account for a 
property that is empty string. Would recommend checking for any required 
property and giving an explicit error message if not present.

##########
File path: 
nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/nar/hadoop/HDFSNarProvider.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nar.hadoop;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.nar.NarProvider;
+import org.apache.nifi.nar.NarProviderInitializationContext;
+import org.apache.nifi.nar.hadoop.util.ExtensionFilter;
+import org.apache.nifi.processors.hadoop.ExtendedConfiguration;
+import org.apache.nifi.processors.hadoop.HdfsResources;
+import org.apache.nifi.security.krb.KerberosKeytabUser;
+import org.apache.nifi.security.krb.KerberosPasswordUser;
+import org.apache.nifi.security.krb.KerberosUser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.SocketFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class HDFSNarProvider implements NarProvider {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(HDFSNarProvider.class);
+
+    private static final String RESOURCES_PARAMETER = "resources";
+    private static final String SOURCE_DIRECTORY_PARAMETER = 
"source.directory";
+    private static final String KERBEROS_PRINCIPAL_PARAMETER = 
"kerberos.principal";
+    private static final String KERBEROS_KEYTAB_PARAMETER = "kerberos.keytab";
+    private static final String KERBEROS_PASSWORD_PARAMETER = 
"kerberos.password";
+
+    private static final String NAR_EXTENSION = "nar";
+    private static final String DELIMITER = "/";
+    private static final int BUFFER_SIZE_DEFAULT = 4096;
+    private static final Object RESOURCES_LOCK = new Object();
+
+    private volatile List<String> resources = null;
+    private volatile Path sourceDirectory = null;
+
+    private volatile NarProviderInitializationContext context;
+
+    private volatile boolean initialized = false;
+
+    public void initialize(final NarProviderInitializationContext context) {
+        resources = 
Arrays.stream(Objects.requireNonNull(context.getParameters().get(RESOURCES_PARAMETER)).split(",")).map(s
 -> s.trim()).collect(Collectors.toList());

Review comment:
       I can envision users accidentally making a typo that includes 2 commas 
in a row, and having a lot of trouble understanding what went wrong when they 
see odd error messages. Recommend adding a `.filter(s -> !s.isEmpty())` before 
the collect

##########
File path: 
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarProviderTask.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.nar;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+final class NarProviderTask implements Runnable {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(NarProviderTask.class);
+    private static final String NAR_EXTENSION = "nar";
+
+    // A unique id is necessary for temporary files not to collide with 
temporary files from other instances.
+    private final String id = UUID.randomUUID().toString();
+
+    private final NarProvider narProvider;
+    private final long pollTimeInMs;
+    private final File extensionDirectory;
+
+    private volatile boolean stopped = false;
+
+    NarProviderTask(final NarProvider narProvider, final File 
extensionDirectory, final long pollTimeInMs) {
+        this.narProvider = narProvider;
+        this.pollTimeInMs = pollTimeInMs;
+        this.extensionDirectory = extensionDirectory;
+    }
+
+    @Override
+    public void run() {
+        LOGGER.info("Nar provider task is started");
+
+        while (!stopped) {
+            try {
+                LOGGER.debug("Task starts fetching NARs from provider");
+                final Set<String> loadedNars = getLoadedNars();
+                final Collection<String> availableNars = 
narProvider.listNars();

Review comment:
       We need to ensure that any invocation of a method in `NarProvider` is 
properly encapsulated in the appropriate ClassLoader. Easiest way to do that is 
to wrap the calls in NarCloseable:
   ```
   final Collection<String> availableNars;
   try (final NarCloseable narCloseable = NarCloseable.withNarLoader() {
       availableNars = narProvider.listNars();
   }
   ```

##########
File path: 
nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/nar/hadoop/HDFSNarProvider.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nar.hadoop;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.nar.NarProvider;
+import org.apache.nifi.nar.NarProviderInitializationContext;
+import org.apache.nifi.nar.hadoop.util.ExtensionFilter;
+import org.apache.nifi.processors.hadoop.ExtendedConfiguration;
+import org.apache.nifi.processors.hadoop.HdfsResources;
+import org.apache.nifi.security.krb.KerberosKeytabUser;
+import org.apache.nifi.security.krb.KerberosPasswordUser;
+import org.apache.nifi.security.krb.KerberosUser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.SocketFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class HDFSNarProvider implements NarProvider {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(HDFSNarProvider.class);
+
+    private static final String RESOURCES_PARAMETER = "resources";
+    private static final String SOURCE_DIRECTORY_PARAMETER = 
"source.directory";
+    private static final String KERBEROS_PRINCIPAL_PARAMETER = 
"kerberos.principal";
+    private static final String KERBEROS_KEYTAB_PARAMETER = "kerberos.keytab";
+    private static final String KERBEROS_PASSWORD_PARAMETER = 
"kerberos.password";
+
+    private static final String NAR_EXTENSION = "nar";
+    private static final String DELIMITER = "/";
+    private static final int BUFFER_SIZE_DEFAULT = 4096;
+    private static final Object RESOURCES_LOCK = new Object();
+
+    private volatile List<String> resources = null;
+    private volatile Path sourceDirectory = null;
+
+    private volatile NarProviderInitializationContext context;
+
+    private volatile boolean initialized = false;
+
+    public void initialize(final NarProviderInitializationContext context) {
+        resources = 
Arrays.stream(Objects.requireNonNull(context.getParameters().get(RESOURCES_PARAMETER)).split(",")).map(s
 -> s.trim()).collect(Collectors.toList());
+
+        if (resources.isEmpty()) {
+            throw new IllegalArgumentException("At least one HDFS 
configuration resource is necessary");
+        }
+
+        this.sourceDirectory = new 
Path(Objects.requireNonNull(context.getParameters().get(SOURCE_DIRECTORY_PARAMETER)));
+        this.context = context;
+        this.initialized = true;
+    }
+
+    @Override
+    public Collection<String> listNars() throws IOException {
+        if (!initialized) {
+            LOGGER.error("Provider is not initialized");
+        }
+
+        final HdfsResources hdfsResources = getHdfsResources();
+        final FileStatus[] fileStatuses = 
hdfsResources.getFileSystem().listStatus(sourceDirectory, new 
ExtensionFilter(NAR_EXTENSION));

Review comment:
       Any calls to the FileSystem like this need to be wrapped with a 
UserGroupInformation.doAs, just as you've done on line 121/122.

##########
File path: 
nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/nar/hadoop/HDFSNarProvider.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nar.hadoop;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.nar.NarProvider;
+import org.apache.nifi.nar.NarProviderInitializationContext;
+import org.apache.nifi.nar.hadoop.util.ExtensionFilter;
+import org.apache.nifi.processors.hadoop.ExtendedConfiguration;
+import org.apache.nifi.processors.hadoop.HdfsResources;
+import org.apache.nifi.security.krb.KerberosKeytabUser;
+import org.apache.nifi.security.krb.KerberosPasswordUser;
+import org.apache.nifi.security.krb.KerberosUser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.SocketFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class HDFSNarProvider implements NarProvider {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(HDFSNarProvider.class);
+
+    private static final String RESOURCES_PARAMETER = "resources";
+    private static final String SOURCE_DIRECTORY_PARAMETER = 
"source.directory";
+    private static final String KERBEROS_PRINCIPAL_PARAMETER = 
"kerberos.principal";
+    private static final String KERBEROS_KEYTAB_PARAMETER = "kerberos.keytab";
+    private static final String KERBEROS_PASSWORD_PARAMETER = 
"kerberos.password";
+
+    private static final String NAR_EXTENSION = "nar";
+    private static final String DELIMITER = "/";
+    private static final int BUFFER_SIZE_DEFAULT = 4096;
+    private static final Object RESOURCES_LOCK = new Object();
+
+    private volatile List<String> resources = null;
+    private volatile Path sourceDirectory = null;
+
+    private volatile NarProviderInitializationContext context;
+
+    private volatile boolean initialized = false;
+
+    public void initialize(final NarProviderInitializationContext context) {
+        resources = 
Arrays.stream(Objects.requireNonNull(context.getParameters().get(RESOURCES_PARAMETER)).split(",")).map(s
 -> s.trim()).collect(Collectors.toList());
+
+        if (resources.isEmpty()) {
+            throw new IllegalArgumentException("At least one HDFS 
configuration resource is necessary");
+        }
+
+        this.sourceDirectory = new 
Path(Objects.requireNonNull(context.getParameters().get(SOURCE_DIRECTORY_PARAMETER)));
+        this.context = context;
+        this.initialized = true;
+    }
+
+    @Override
+    public Collection<String> listNars() throws IOException {
+        if (!initialized) {
+            LOGGER.error("Provider is not initialized");
+        }
+
+        final HdfsResources hdfsResources = getHdfsResources();
+        final FileStatus[] fileStatuses = 
hdfsResources.getFileSystem().listStatus(sourceDirectory, new 
ExtensionFilter(NAR_EXTENSION));
+
+        final List<String> result = Arrays.stream(fileStatuses)
+            .filter(fileStatus -> fileStatus.isFile())
+            .map(fileStatus -> fileStatus.getPath().getName())
+            .collect(Collectors.toList());
+
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("The following nars were found: " + String.join(", ", 
result));
+        }
+
+        return result;
+    }
+
+    @Override
+    public InputStream fetchNarContents(final String location) throws 
IOException {
+        if (!initialized) {
+            LOGGER.error("Provider is not initialized");
+        }
+
+
+        final Path path = getNarLocation(location);
+        final HdfsResources hdfsResources = getHdfsResources();
+
+        if (!hdfsResources.getFileSystem().exists(path)) {
+            throw new IOException("Provider cannot find " + location);
+        }
+
+        try {
+            return hdfsResources.getUserGroupInformation()
+                .doAs((PrivilegedExceptionAction<FSDataInputStream>) () -> 
hdfsResources.getFileSystem().open(path, BUFFER_SIZE_DEFAULT));
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            LOGGER.error("Error during acquiring file", e);
+            throw new RuntimeException();
+        }
+    }
+
+    private Path getNarLocation(final String location) {
+        String result = sourceDirectory.toString();
+
+        if (!result.endsWith(DELIMITER)) {
+            result += DELIMITER;
+        }
+
+        return new Path(result + location);
+    }
+
+    private HdfsResources getHdfsResources() throws IOException {
+        final Configuration config = new ExtendedConfiguration(LOGGER);
+        config.setClassLoader(this.getClass().getClassLoader());
+
+        for (final String resource : resources) {
+            config.addResource(new Path(resource));
+        }
+
+        // first check for timeout on HDFS connection, because FileSystem has 
a hard coded 15 minute timeout
+        checkHdfsUriForTimeout(config);
+
+        // disable caching of Configuration and FileSystem objects, else we 
cannot reconfigure the processor without a complete restart
+        final String disableCacheName = 
String.format("fs.%s.impl.disable.cache", 
FileSystem.getDefaultUri(config).getScheme());
+        config.set(disableCacheName, "true");
+
+        // If kerberos is enabled, create the file system as the kerberos 
principal
+        // -- use RESOURCE_LOCK to guarantee UserGroupInformation is accessed 
by only a single thread at at time
+        FileSystem fs;
+        UserGroupInformation ugi;
+        KerberosUser kerberosUser;
+
+        synchronized (RESOURCES_LOCK) {
+            if (SecurityUtil.isSecurityEnabled(config)) {
+                final String principal = 
context.getParameters().get(KERBEROS_PRINCIPAL_PARAMETER);
+                final String keyTab = 
context.getParameters().get(KERBEROS_KEYTAB_PARAMETER);
+                final String password = 
context.getParameters().get(KERBEROS_PASSWORD_PARAMETER);
+
+                if (keyTab != null) {
+                    kerberosUser = new KerberosKeytabUser(principal, keyTab);
+                } else if (password != null) {
+                    kerberosUser = new KerberosPasswordUser(principal, 
password);
+                } else {
+                    throw new IOException("Unable to authenticate with 
Kerberos, no keytab or password was provided");
+                }
+
+                ugi = SecurityUtil.getUgiForKerberosUser(config, kerberosUser);
+            } else {
+                config.set("ipc.client.fallback-to-simple-auth-allowed", 
"true");
+                config.set("hadoop.security.authentication", "simple");
+                ugi = SecurityUtil.loginSimple(config);
+                kerberosUser = null;
+            }
+
+            fs = getFileSystemAsUser(config, ugi);
+        }
+        LOGGER.debug("resetHDFSResources UGI [{}], KerberosUser [{}]", new 
Object[]{ugi, kerberosUser});
+
+        final Path workingDir = fs.getWorkingDirectory();
+        LOGGER.debug("Initialized a new HDFS File System with working dir: {} 
default block size: {} default replication: {} config: {}",
+                new Object[]{workingDir, fs.getDefaultBlockSize(workingDir), 
fs.getDefaultReplication(workingDir), config.toString()});
+
+        if (!fs.exists(sourceDirectory)) {
+            throw new IllegalArgumentException("Source directory is not 
existing");
+        }
+
+        return new HdfsResources(config, fs, ugi, kerberosUser);
+    }
+
+    private void checkHdfsUriForTimeout(final Configuration config) throws 
IOException {
+        final URI hdfsUri = FileSystem.getDefaultUri(config);
+        final String address = hdfsUri.getAuthority();
+        final int port = hdfsUri.getPort();
+
+        if (address == null || address.isEmpty() || port < 0) {
+            return;
+        }
+
+        final InetSocketAddress namenode = NetUtils.createSocketAddr(address, 
port);
+        final SocketFactory socketFactory = 
NetUtils.getDefaultSocketFactory(config);
+        Socket socket = null;
+
+        try {

Review comment:
       This can be made into a try-with-resources, no?

##########
File path: 
nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/nar/hadoop/HDFSNarProvider.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nar.hadoop;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.nar.NarProvider;
+import org.apache.nifi.nar.NarProviderInitializationContext;
+import org.apache.nifi.nar.hadoop.util.ExtensionFilter;
+import org.apache.nifi.processors.hadoop.ExtendedConfiguration;
+import org.apache.nifi.processors.hadoop.HdfsResources;
+import org.apache.nifi.security.krb.KerberosKeytabUser;
+import org.apache.nifi.security.krb.KerberosPasswordUser;
+import org.apache.nifi.security.krb.KerberosUser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.SocketFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class HDFSNarProvider implements NarProvider {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(HDFSNarProvider.class);
+
+    private static final String RESOURCES_PARAMETER = "resources";
+    private static final String SOURCE_DIRECTORY_PARAMETER = 
"source.directory";
+    private static final String KERBEROS_PRINCIPAL_PARAMETER = 
"kerberos.principal";
+    private static final String KERBEROS_KEYTAB_PARAMETER = "kerberos.keytab";
+    private static final String KERBEROS_PASSWORD_PARAMETER = 
"kerberos.password";
+
+    private static final String NAR_EXTENSION = "nar";
+    private static final String DELIMITER = "/";
+    private static final int BUFFER_SIZE_DEFAULT = 4096;
+    private static final Object RESOURCES_LOCK = new Object();
+
+    private volatile List<String> resources = null;
+    private volatile Path sourceDirectory = null;
+
+    private volatile NarProviderInitializationContext context;
+
+    private volatile boolean initialized = false;
+
+    public void initialize(final NarProviderInitializationContext context) {
+        resources = 
Arrays.stream(Objects.requireNonNull(context.getParameters().get(RESOURCES_PARAMETER)).split(",")).map(s
 -> s.trim()).collect(Collectors.toList());
+
+        if (resources.isEmpty()) {
+            throw new IllegalArgumentException("At least one HDFS 
configuration resource is necessary");
+        }
+
+        this.sourceDirectory = new 
Path(Objects.requireNonNull(context.getParameters().get(SOURCE_DIRECTORY_PARAMETER)));
+        this.context = context;
+        this.initialized = true;
+    }
+
+    @Override
+    public Collection<String> listNars() throws IOException {
+        if (!initialized) {
+            LOGGER.error("Provider is not initialized");
+        }
+
+        final HdfsResources hdfsResources = getHdfsResources();
+        final FileStatus[] fileStatuses = 
hdfsResources.getFileSystem().listStatus(sourceDirectory, new 
ExtensionFilter(NAR_EXTENSION));
+
+        final List<String> result = Arrays.stream(fileStatuses)
+            .filter(fileStatus -> fileStatus.isFile())
+            .map(fileStatus -> fileStatus.getPath().getName())
+            .collect(Collectors.toList());
+
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("The following nars were found: " + String.join(", ", 
result));
+        }
+
+        return result;
+    }
+
+    @Override
+    public InputStream fetchNarContents(final String location) throws 
IOException {
+        if (!initialized) {
+            LOGGER.error("Provider is not initialized");
+        }
+
+
+        final Path path = getNarLocation(location);
+        final HdfsResources hdfsResources = getHdfsResources();
+
+        if (!hdfsResources.getFileSystem().exists(path)) {
+            throw new IOException("Provider cannot find " + location);
+        }
+
+        try {
+            return hdfsResources.getUserGroupInformation()
+                .doAs((PrivilegedExceptionAction<FSDataInputStream>) () -> 
hdfsResources.getFileSystem().open(path, BUFFER_SIZE_DEFAULT));
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            LOGGER.error("Error during acquiring file", e);
+            throw new RuntimeException();
+        }
+    }
+
+    private Path getNarLocation(final String location) {
+        String result = sourceDirectory.toString();
+
+        if (!result.endsWith(DELIMITER)) {
+            result += DELIMITER;
+        }
+
+        return new Path(result + location);
+    }
+
+    private HdfsResources getHdfsResources() throws IOException {
+        final Configuration config = new ExtendedConfiguration(LOGGER);
+        config.setClassLoader(this.getClass().getClassLoader());

Review comment:
       I think we want to use `Thread.currentThread().getContextClassLoader()` 
rather than `getClass().getClassLoader()`.

##########
File path: 
nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/nar/hadoop/HDFSNarProvider.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nar.hadoop;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.nar.NarProvider;
+import org.apache.nifi.nar.NarProviderInitializationContext;
+import org.apache.nifi.nar.hadoop.util.ExtensionFilter;
+import org.apache.nifi.processors.hadoop.ExtendedConfiguration;
+import org.apache.nifi.processors.hadoop.HdfsResources;
+import org.apache.nifi.security.krb.KerberosKeytabUser;
+import org.apache.nifi.security.krb.KerberosPasswordUser;
+import org.apache.nifi.security.krb.KerberosUser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.SocketFactory;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+public class HDFSNarProvider implements NarProvider {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(HDFSNarProvider.class);
+
+    private static final String RESOURCES_PARAMETER = "resources";
+    private static final String SOURCE_DIRECTORY_PARAMETER = 
"source.directory";
+    private static final String KERBEROS_PRINCIPAL_PARAMETER = 
"kerberos.principal";
+    private static final String KERBEROS_KEYTAB_PARAMETER = "kerberos.keytab";
+    private static final String KERBEROS_PASSWORD_PARAMETER = 
"kerberos.password";
+
+    private static final String NAR_EXTENSION = "nar";
+    private static final String DELIMITER = "/";
+    private static final int BUFFER_SIZE_DEFAULT = 4096;
+    private static final Object RESOURCES_LOCK = new Object();
+
+    private volatile List<String> resources = null;
+    private volatile Path sourceDirectory = null;
+
+    private volatile NarProviderInitializationContext context;
+
+    private volatile boolean initialized = false;
+
+    public void initialize(final NarProviderInitializationContext context) {
+        resources = 
Arrays.stream(Objects.requireNonNull(context.getParameters().get(RESOURCES_PARAMETER)).split(",")).map(s
 -> s.trim()).collect(Collectors.toList());
+
+        if (resources.isEmpty()) {
+            throw new IllegalArgumentException("At least one HDFS 
configuration resource is necessary");
+        }
+
+        this.sourceDirectory = new 
Path(Objects.requireNonNull(context.getParameters().get(SOURCE_DIRECTORY_PARAMETER)));
+        this.context = context;
+        this.initialized = true;
+    }
+
+    @Override
+    public Collection<String> listNars() throws IOException {
+        if (!initialized) {
+            LOGGER.error("Provider is not initialized");
+        }
+
+        final HdfsResources hdfsResources = getHdfsResources();
+        final FileStatus[] fileStatuses = 
hdfsResources.getFileSystem().listStatus(sourceDirectory, new 
ExtensionFilter(NAR_EXTENSION));
+
+        final List<String> result = Arrays.stream(fileStatuses)
+            .filter(fileStatus -> fileStatus.isFile())
+            .map(fileStatus -> fileStatus.getPath().getName())
+            .collect(Collectors.toList());
+
+        if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("The following nars were found: " + String.join(", ", 
result));
+        }
+
+        return result;
+    }
+
+    @Override
+    public InputStream fetchNarContents(final String location) throws 
IOException {
+        if (!initialized) {
+            LOGGER.error("Provider is not initialized");
+        }
+
+
+        final Path path = getNarLocation(location);
+        final HdfsResources hdfsResources = getHdfsResources();
+
+        if (!hdfsResources.getFileSystem().exists(path)) {

Review comment:
       Any calls to the FileSystem like this need to be wrapped with a 
UserGroupInformation.doAs, just as you've done on line 121/122.

##########
File path: 
nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarProviderTask.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.nar;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+final class NarProviderTask implements Runnable {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(NarProviderTask.class);
+    private static final String NAR_EXTENSION = "nar";
+
+    // A unique id is necessary for temporary files not to collide with 
temporary files from other instances.
+    private final String id = UUID.randomUUID().toString();
+
+    private final NarProvider narProvider;
+    private final long pollTimeInMs;
+    private final File extensionDirectory;
+
+    private volatile boolean stopped = false;
+
+    NarProviderTask(final NarProvider narProvider, final File 
extensionDirectory, final long pollTimeInMs) {
+        this.narProvider = narProvider;
+        this.pollTimeInMs = pollTimeInMs;
+        this.extensionDirectory = extensionDirectory;
+    }
+
+    @Override
+    public void run() {
+        LOGGER.info("Nar provider task is started");
+
+        while (!stopped) {
+            try {
+                LOGGER.debug("Task starts fetching NARs from provider");
+                final Set<String> loadedNars = getLoadedNars();
+                final Collection<String> availableNars = 
narProvider.listNars();
+
+                for (final String availableNar : availableNars) {
+                    if (!loadedNars.contains(availableNar)) {
+                        final long startedAt = System.currentTimeMillis();
+                        final InputStream inputStream = 
narProvider.fetchNarContents(availableNar);
+                        // There is a random uuid for the case when multiple 
providers are in place
+                        final File tempFile = new File(extensionDirectory + 
File.separator + ".tmp_" + id + ".nar");
+                        final File targetFile = new File(extensionDirectory + 
File.separator + availableNar);
+                        Files.copy(inputStream, tempFile.toPath(), 
StandardCopyOption.REPLACE_EXISTING);
+                        tempFile.renameTo(targetFile);
+
+                        LOGGER.info("Nar {} is opened in {}ms", availableNar, 
(System.currentTimeMillis() - startedAt));

Review comment:
       Would recommend phrasing as "Downloaded NAR {} in {} ms"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to