You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/09/01 19:51:58 UTC

[GitHub] [nifi] tpalfy opened a new pull request, #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

tpalfy opened a new pull request, #6355:
URL: https://github.com/apache/nifi/pull/6355

   <!-- 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. -->
   
   # Summary
   
   [NIFI-10427](https://issues.apache.org/jira/browse/NIFI-10427)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] turcsanyip commented on pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on PR #6355:
URL: https://github.com/apache/nifi/pull/6355#issuecomment-1250810447

   @tpalfy Could you please rename the processors to `ListBoxFile` / `FetchBoxFile` because processor names should not use plurals by convention.
   
   Please also rebase you branch onto main and then I am ready to merge it.
   Thanks


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r970711367


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/pom.xml:
##########
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>

Review Comment:
   The resources/META-INF.services/org.apache.nifi.controller.ControllerService file is missing.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/src/main/java/org/apache/nifi/box/controllerservices/JsonConfigBasedBoxClientService.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.box.controllerservices;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.JsonValidator;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+public class JsonConfigBasedBoxClientService extends AbstractControllerService implements BoxClientService {

Review Comment:
   @CapabilityDescription and @Tags are missing.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/src/main/resources/docs/org.apache.nifi.box.controllerservices.JsonConfigBasedBoxClientService/additionalDetails.html:
##########
@@ -0,0 +1,46 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>

Review Comment:
   Ultra minor comment: the title remained ListBoxFiles, pls change it to the name of controller service.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-nar/pom.xml:
##########
@@ -0,0 +1,78 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-box-bundle</artifactId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-box-nar</artifactId>
+    <packaging>nar</packaging>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>
+
+    <dependencies>
+        <dependency>

Review Comment:
   Reference to nifi-box-services-api-nar is missing.
   
   When nifi-box-services-api-nar dependency is added, nifi-standard-services-api-nar dependency can be removed since nifi-box-services-api-nar contains it.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/pom.xml:
##########
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>nifi-box-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-box-services</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-box-services-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-json-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>

Review Comment:
   The scope can be "test" for nifi-mock.
   The compile scopes can be removed since compile is the default scope.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r966153514


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/BoxFlowFileAttribute.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import java.util.Optional;
+import java.util.function.Function;
+
+public enum BoxFlowFileAttribute {
+    ID(BoxFileInfo.ID, BoxFileInfo::getId),
+    FILE_NAME(BoxFileInfo.FILENAME, BoxFileInfo::getName),

Review Comment:
   Consider changing the enum names to FILENAME and TIMESTAMP.
   The Optional.ofNullable check is not necessary for size and timestamp since they have long value.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/test/java/org/apache/nifi/processors/box/AbstractBoxFilesIT.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Set the following constants before running:<br />
+ * <br />
+ * FOLDER_ID - The ID of a Folder the test can use to create files and sub-folders within.<br />
+ * USER_ID - The ID of the user owning the Folder.<br />
+ * BOX_CONFIG_FILE - An App Settings Configuration JSON file. The app needs to have 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' authentication method,
+ * have App + Enterprise access level and be able to read and write files and folders as well as generate user access tokens.<br />
+ * <br />
+ * Created files and folders are cleaned up, but it's advisable to dedicate a folder for this test so that it can be cleaned up easily should the test fail to do so.
+ */
+public abstract class AbstractBoxFilesIT<T extends BoxTrait & Processor> {
+    static final String FOLDER_ID = "";
+    static final String USER_ID = "";
+    static final String BOX_CONFIG_FILE = "";
+
+    protected static final String DEFAULT_FILE_CONTENT = "test_content";
+    public static final String MAIN_FOLDER_NAME = "main";
+
+    protected T testSubject;
+    protected TestRunner testRunner;
+
+    protected BoxAPIConnection boxAPIConnection;
+
+    protected String targetFolderName;
+    protected String mainFolderId;
+
+    protected abstract T createTestSubject();
+
+    @BeforeEach
+    protected void init() throws Exception {
+        testSubject = createTestSubject();
+        testRunner = createTestRunner();
+
+        try (
+            Reader reader = new FileReader(BOX_CONFIG_FILE);
+        ) {
+            BoxConfig boxConfig = BoxConfig.readFrom(reader);
+            boxAPIConnection = BoxDeveloperEditionAPIConnection.getAppEnterpriseConnection(boxConfig);
+            boxAPIConnection.asUser(USER_ID);
+        }
+
+        targetFolderName = new BoxFolder(boxAPIConnection, FOLDER_ID).getInfo("name").getName();
+
+        BoxFolder.Info mainFolderInfo = createFolder(MAIN_FOLDER_NAME, FOLDER_ID);
+        mainFolderId = mainFolderInfo.getID();
+    }
+
+    @AfterEach
+    protected void tearDown() {
+        if (boxAPIConnection != null) {
+            BoxFolder folder = new BoxFolder(boxAPIConnection, mainFolderId);
+            folder.delete(true);
+        }
+    }
+
+    protected TestRunner createTestRunner() {
+        TestRunner testRunner = TestRunners.newTestRunner(testSubject);

Review Comment:
   Minor comment: the local variable is not necessary (same comment for createFolder, createFile and getCheckedAttributeNames)



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/pom.xml:
##########
@@ -0,0 +1,121 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-box-bundle</artifactId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-box-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.box</groupId>
+            <artifactId>box-java-sdk</artifactId>
+            <version>3.4.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-proxy-configuration-api</artifactId>
+        </dependency>
+        <dependency>

Review Comment:
   Some of these dependencies are used only by the tests (e.g. nifi-record-serialization-service-api, nifi-schema-registry-service-api, nifi-distributed-cache-client-service-api).



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/test/java/org/apache/nifi/processors/box/ListBoxFilesSimpleTest.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxFolder;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.util.EqualsWrapper;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.nifi.util.EqualsWrapper.wrapList;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Answers.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+
+public class ListBoxFilesSimpleTest implements SimpleListBoxFileTestTrait {
+    private ListBoxFiles testSubject;
+
+    private ProcessContext mockProcessContext;
+    private BoxAPIConnection mockBoxAPIConnection;
+
+    private BoxFolder mockBoxFolder;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        mockProcessContext = mock(ProcessContext.class, RETURNS_DEEP_STUBS);
+        mockBoxAPIConnection = mock(BoxAPIConnection.class, RETURNS_DEEP_STUBS);
+        mockBoxFolder = mock(BoxFolder.class, RETURNS_DEEP_STUBS);
+
+        testSubject = new ListBoxFiles() {
+            @Override
+            protected List<BoxFileInfo> performListing(ProcessContext context, Long minTimestamp, AbstractListProcessor.ListingMode ignoredListingMode) throws IOException {
+                return super.performListing(context, minTimestamp, ListingMode.EXECUTION);
+            }
+
+            @Override
+            public BoxAPIConnection createBoxApiConnection(ProcessContext context, ProxyConfiguration proxyConfiguration) {
+                return mockBoxAPIConnection;
+            }
+
+            @Override
+            BoxFolder getFolder(String folderId) {
+                return mockBoxFolder;
+            }
+        };
+
+        testSubject.onScheduled(mockProcessContext);
+    }
+
+    @Test
+    void testCreatedListableEntityContainsCorrectData() throws Exception {
+        // GIVEN
+        Long minTimestamp = 0L;
+
+        String id = "id_1";
+        String filename = "file_name_1";
+        List<String> pathParts = Arrays.asList("path", "to", "file");
+        Long size = 125L;
+        long createdTime = 123456L;
+        long modifiedTime = 234567L;
+
+        mockFetchedFileList(id, filename, pathParts, size, createdTime, modifiedTime);
+
+        List<BoxFileInfo> expected = Arrays.asList(
+            new BoxFileInfo.Builder()
+                .id(id)
+                .fileName(filename)
+                .path("/path/to/file")
+                .size(size)
+                .createdTime(createdTime)
+                .modifiedTime(modifiedTime)
+                .build()
+        );
+
+        // WHEN
+        List<BoxFileInfo> actual = testSubject.performListing(mockProcessContext, minTimestamp, null);
+
+        // THEN
+        List<Function<BoxFileInfo, Object>> propertyProviders = Arrays.asList(
+            BoxFileInfo::getId,
+            BoxFileInfo::getIdentifier,
+            BoxFileInfo::getName,
+            BoxFileInfo::getSize,

Review Comment:
   getPath is missing from propertyProviders



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/FetchBoxFiles.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxAPIResponseException;
+import com.box.sdk.BoxFile;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"box", "storage", "fetch"})
+@CapabilityDescription("Fetches files from a Box Folder. Designed to be used in tandem with ListBoxFiles.")
+@SeeAlso({ListBoxFiles.class})
+@WritesAttributes({
+    @WritesAttribute(attribute = FetchBoxFiles.ERROR_CODE_ATTRIBUTE, description = "The error code returned by Box when the fetch of a file fails"),
+    @WritesAttribute(attribute = FetchBoxFiles.ERROR_MESSAGE_ATTRIBUTE, description = "The error message returned by Box when the fetch of a file fails")
+})
+public class FetchBoxFiles extends AbstractProcessor implements BoxTrait {
+    public static final String ERROR_CODE_ATTRIBUTE = "error.code";
+    public static final String ERROR_MESSAGE_ATTRIBUTE = "error.message";
+
+    public static final PropertyDescriptor FILE_ID = new PropertyDescriptor
+        .Builder().name("box-file-id")
+        .displayName("File ID")
+        .description("The ID of the File to fetch")
+        .required(true)
+        .defaultValue("${box.id}")
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
+    public static final Relationship REL_SUCCESS =
+        new Relationship.Builder()
+            .name("success")
+            .description("A flowfile will be routed here for each successfully fetched File.")
+            .build();
+
+    public static final Relationship REL_FAILURE =
+        new Relationship.Builder().name("failure")
+            .description("A flowfile will be routed here for each File for which fetch was attempted but failed.")
+            .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+        FILE_ID,
+        USER_ID,
+        BOX_CONFIG_FILE,
+        ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+    ));
+
+    public static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+        REL_SUCCESS,
+        REL_FAILURE
+    )));
+
+    private volatile BoxAPIConnection boxAPIConnection;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws IOException {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+
+        boxAPIConnection = createBoxApiConnection(context, proxyConfiguration);
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String fileId = context.getProperty(FILE_ID).evaluateAttributeExpressions(flowFile).getValue();
+        FlowFile outFlowFile = flowFile;
+        try {
+            outFlowFile = fetchFile(fileId, session, outFlowFile);
+
+            session.transfer(outFlowFile, REL_SUCCESS);
+        } catch (BoxAPIResponseException e) {
+            handleErrorResponse(session, fileId, flowFile, e);
+        } catch (Exception e) {
+            handleUnexpectedError(session, flowFile, fileId, e);
+        }
+    }
+
+    FlowFile fetchFile(String fileId, ProcessSession session, FlowFile outFlowFile) {
+        BoxFile boxFile = new BoxFile(boxAPIConnection, fileId);
+
+        outFlowFile = session.write(outFlowFile, outputStream -> boxFile.download(outputStream));
+
+        return outFlowFile;
+    }
+
+    private void handleErrorResponse(ProcessSession session, String fileId, FlowFile outFlowFile, BoxAPIResponseException e) {
+        getLogger().error("Couldn't fetch file with id '{}'", fileId, e);
+
+        session.putAttribute(outFlowFile, ERROR_CODE_ATTRIBUTE, "" + e.getResponseCode());

Review Comment:
   Please use the return value of putAttribute.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r968478534


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.FetchBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,36 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the files that are being fetched.</p>
+<p>The App should have the following configuration:</p>
+<ul>
+    <li>If you create a new App, select 'Server Authentication (with JWT)' authentication method.
+        If you want to use an existing App, chose one with 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' as Authentication method.</li>
+    <li>Should have a Client ID and Client Secret</li>
+    <li>App Access Level should be 'App + Enterprise Access'</li>
+    <li>Application Scopes should have 'Read all files' enabled</li>

Review Comment:
   Read access is not enough, download requires write access, please change this line to:
   "Application Scopes should have 'Write all files and folders in Box' enabled."



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.FetchBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,36 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the files that are being fetched.</p>
+<p>The App should have the following configuration:</p>
+<ul>
+    <li>If you create a new App, select 'Server Authentication (with JWT)' authentication method.
+        If you want to use an existing App, chose one with 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' as Authentication method.</li>
+    <li>Should have a Client ID and Client Secret</li>
+    <li>App Access Level should be 'App + Enterprise Access'</li>
+    <li>Application Scopes should have 'Read all files' enabled</li>
+    <li>Advanced Features should have 'Generate user access tokens' enabled</li>

Review Comment:
   Besides 'Generate user access tokens', the 'Make API calls using the as-user header" needs to be enabled, otherwise the API returns 403 error.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/ListBoxFiles.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import com.box.sdk.BoxItem;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"box", "storage"})
+@CapabilityDescription("Lists files in a Box folder. " +
+    "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+    "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile. " +
+    "This Processor is designed to run on Primary Node only in a cluster. If the primary node changes, the new Primary Node will pick up where the " +
+    "previous node left off without duplicating all of the data.")
+@SeeAlso({FetchBoxFiles.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = BoxFileInfo.ID, description = "The id of the file"),
+    @WritesAttribute(attribute = BoxFileInfo.FILENAME, description = "The name of the file"),
+    @WritesAttribute(attribute = BoxFileInfo.PATH, description = "The path of the file on Box"),
+    @WritesAttribute(attribute = BoxFileInfo.SIZE, description = "The size of the file (in bytes)"),
+    @WritesAttribute(attribute = BoxFileInfo.TIMESTAMP, description = "The last modified time of the file.")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "The processor stores necessary data to be able to keep track what files have been listed already." +
+    " What exactly needs to be stored depends on the 'Listing Strategy'.")
+public class ListBoxFiles extends AbstractListProcessor<BoxFileInfo> implements BoxTrait {
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+        .name("box-folder-id")
+        .displayName("Folder ID")
+        .description("The ID of the folder from which to pull list of files.")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .required(true)
+        .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+        .name("recursive-search")
+        .displayName("Search Recursively")
+        .description("When 'true', will include list of files from sub-folders." +
+            " Otherwise, will return only files that are within the folder defined by the 'Folder ID' property.")
+        .required(true)
+        .defaultValue("true")
+        .allowableValues("true", "false")
+        .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+        .name("min-age")
+        .displayName("Minimum File Age")
+        .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored.")
+        .required(true)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+
+    public static final PropertyDescriptor LISTING_STRATEGY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(AbstractListProcessor.LISTING_STRATEGY)
+        .allowableValues(BY_TIMESTAMPS, BY_ENTITIES, BY_TIME_WINDOW, NO_TRACKING)
+        .build();
+
+    public static final PropertyDescriptor TRACKING_STATE_CACHE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.TRACKING_STATE_CACHE)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    public static final PropertyDescriptor TRACKING_TIME_WINDOW = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.TRACKING_TIME_WINDOW)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    public static final PropertyDescriptor INITIAL_LISTING_TARGET = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.INITIAL_LISTING_TARGET)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+        FOLDER_ID,
+        USER_ID,
+        BOX_CONFIG_FILE,
+        RECURSIVE_SEARCH,
+        MIN_AGE,
+        LISTING_STRATEGY,
+        TRACKING_STATE_CACHE,
+        TRACKING_TIME_WINDOW,
+        INITIAL_LISTING_TARGET,
+        RECORD_WRITER,
+        ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+    ));
+
+    private volatile BoxAPIConnection boxAPIConnection;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext validationContext, Collection<ValidationResult> results) {

Review Comment:
   This method can be removed since it does not override the parent implementation (which is also an empty method)



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/test/java/org/apache/nifi/processors/box/ListBoxFilesIT.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxFolder;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * See Javadoc {@link AbstractBoxFilesIT} for instructions how to run this test.
+ */
+public class ListBoxFilesIT extends AbstractBoxFilesIT<ListBoxFiles> {
+    @BeforeEach
+    public void init() throws Exception {
+        super.init();
+        testRunner.setProperty(ListBoxFiles.FOLDER_ID, mainFolderId);
+        testRunner.setProperty(ListBoxFiles.USER_ID, USER_ID);
+        testRunner.setProperty(ListBoxFiles.BOX_CONFIG_FILE, BOX_CONFIG_FILE);
+    }
+
+    @Override
+    public ListBoxFiles createTestSubject() {
+        ListBoxFiles testSubject = new ListBoxFiles();
+
+        return testSubject;
+    }
+
+    @Test
+    void listFilesFrom3LayerDeepDirectoryTree() throws Exception {
+        // GIVEN
+        BoxFolder.Info main_sub1 = createFolder("main_sub1", mainFolderId);
+        BoxFolder.Info main_sub2 = createFolder("main_sub2", mainFolderId);
+
+        BoxFolder.Info main_sub1_sub1 = createFolder("main_sub1_sub1", main_sub1.getID());
+
+        createFileWithDefaultContent("main_file1", mainFolderId);
+        createFileWithDefaultContent("main_file2", mainFolderId);
+        createFileWithDefaultContent("main_file3", mainFolderId);
+
+        createFileWithDefaultContent("main_sub1_file1", main_sub1.getID());
+
+        createFileWithDefaultContent("main_sub2_file1", main_sub2.getID());
+        createFileWithDefaultContent("main_sub2_file2", main_sub2.getID());
+
+        createFileWithDefaultContent("main_sub1_sub1_file1", main_sub1_sub1.getID());
+        createFileWithDefaultContent("main_sub1_sub1_file2", main_sub1_sub1.getID());
+        createFileWithDefaultContent("main_sub1_sub1_file3", main_sub1_sub1.getID());
+
+        Set<String> expectedFileNames = new HashSet<>(Arrays.asList(
+            "main_file1", "main_file2", "main_file3",
+            "main_sub1_file1",
+            "main_sub2_file1", "main_sub2_file2",
+            "main_sub1_sub1_file1", "main_sub1_sub1_file2", "main_sub1_sub1_file3"
+        ));
+
+        // The creation of the files are not (completely) synchronized.
+        Thread.sleep(2000);
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        List<MockFlowFile> successFlowFiles = testRunner.getFlowFilesForRelationship(ListBoxFiles.REL_SUCCESS);
+
+        Set<String> actualFileNames = successFlowFiles.stream()

Review Comment:
   Please consider extracting this call to a method since the same logic is used to collect actual filenames in 4 places.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/test/java/org/apache/nifi/processors/box/AbstractBoxFilesIT.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Set the following constants before running:<br />
+ * <br />
+ * FOLDER_ID - The ID of a Folder the test can use to create files and sub-folders within.<br />
+ * USER_ID - The ID of the user owning the Folder.<br />
+ * BOX_CONFIG_FILE - An App Settings Configuration JSON file. The app needs to have 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' authentication method,

Review Comment:
   Please mention the "Make API calls using the as-user header" here as well.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.FetchBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,36 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>

Review Comment:
   Wrong title, it should be FetchBoxFiles. 



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/ListBoxFiles.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import com.box.sdk.BoxItem;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"box", "storage"})
+@CapabilityDescription("Lists files in a Box folder. " +
+    "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +

Review Comment:
   Please consider using "FlowFile" instead of "flowfile" in descriptions.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.FetchBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,36 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the files that are being fetched.</p>
+<p>The App should have the following configuration:</p>
+<ul>
+    <li>If you create a new App, select 'Server Authentication (with JWT)' authentication method.
+        If you want to use an existing App, chose one with 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' as Authentication method.</li>
+    <li>Should have a Client ID and Client Secret</li>

Review Comment:
   Please mention the private key generation and the application authorization as well.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r966742947


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/test/java/org/apache/nifi/processors/box/AbstractBoxFilesIT.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Set the following constants before running:<br />
+ * <br />
+ * FOLDER_ID - The ID of a Folder the test can use to create files and sub-folders within.<br />
+ * USER_ID - The ID of the user owning the Folder.<br />
+ * BOX_CONFIG_FILE - An App Settings Configuration JSON file. The app needs to have 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' authentication method,
+ * have App + Enterprise access level and be able to read and write files and folders as well as generate user access tokens.<br />
+ * <br />
+ * Created files and folders are cleaned up, but it's advisable to dedicate a folder for this test so that it can be cleaned up easily should the test fail to do so.
+ */
+public abstract class AbstractBoxFilesIT<T extends BoxTrait & Processor> {
+    static final String FOLDER_ID = "";
+    static final String USER_ID = "";
+    static final String BOX_CONFIG_FILE = "";
+
+    protected static final String DEFAULT_FILE_CONTENT = "test_content";
+    public static final String MAIN_FOLDER_NAME = "main";
+
+    protected T testSubject;
+    protected TestRunner testRunner;
+
+    protected BoxAPIConnection boxAPIConnection;
+
+    protected String targetFolderName;
+    protected String mainFolderId;
+
+    protected abstract T createTestSubject();
+
+    @BeforeEach
+    protected void init() throws Exception {
+        testSubject = createTestSubject();
+        testRunner = createTestRunner();
+
+        try (
+            Reader reader = new FileReader(BOX_CONFIG_FILE);
+        ) {
+            BoxConfig boxConfig = BoxConfig.readFrom(reader);
+            boxAPIConnection = BoxDeveloperEditionAPIConnection.getAppEnterpriseConnection(boxConfig);
+            boxAPIConnection.asUser(USER_ID);
+        }
+
+        targetFolderName = new BoxFolder(boxAPIConnection, FOLDER_ID).getInfo("name").getName();
+
+        BoxFolder.Info mainFolderInfo = createFolder(MAIN_FOLDER_NAME, FOLDER_ID);
+        mainFolderId = mainFolderInfo.getID();
+    }
+
+    @AfterEach
+    protected void tearDown() {
+        if (boxAPIConnection != null) {
+            BoxFolder folder = new BoxFolder(boxAPIConnection, mainFolderId);
+            folder.delete(true);
+        }
+    }
+
+    protected TestRunner createTestRunner() {
+        TestRunner testRunner = TestRunners.newTestRunner(testSubject);

Review Comment:
   OK, I see your point. 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r968875962


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.ListBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,42 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the Folder from which files are being listed.</p>
+<p>The App should have the following configuration:</p>

Review Comment:
   It would make sense to mention the URL of the Box developer page where the app can be created: `https://developer.box.com`



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.ListBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,42 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the Folder from which files are being listed.</p>
+<p>The App should have the following configuration:</p>
+<ul>
+    <li>
+        If you create a new App, select 'Server Authentication (with JWT)' authentication method.
+        <br/>If you want to use an existing App, chose one with 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' as Authentication method.
+    </li>
+    <li>Should have a Client ID and Client Secret.</li>
+    <li>App Access Level should be 'App + Enterprise Access'.</li>
+    <li>Application Scopes should have 'Write all files and folders in Box' enabled.</li>
+    <li>Advanced Features should have 'Generate user access tokens' and 'Make API calls using the as-user header' enabled.</li>
+    <li>
+        Under 'Add and Manage Public Keys' Generate a Public/Private Keypair and donwload the configuration JSON file (under App Settings). The full path of this file should be set in the 'Box Config File' property.

Review Comment:
   ```suggestion
           Under 'Add and Manage Public Keys', generate a Public/Private Keypair and download the configuration JSON file (under App Settings). The full path of this file should be set in the 'Box Config File' property.
   ```



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.ListBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,42 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the Folder from which files are being listed.</p>

Review Comment:
   Could you please add a header and optionally section headers to this page. Like in case of `StandardDropboxCredentialService`
   https://github.com/apache/nifi/blob/6d8dda1c03469a0a9725e7df4604907e0ed6a375/nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-services/src/main/resources/docs/org.apache.nifi.services.dropbox.StandardDropboxCredentialService/additionalDetails.html#L25-L37



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.ListBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,42 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the Folder from which files are being listed.</p>
+<p>The App should have the following configuration:</p>
+<ul>
+    <li>
+        If you create a new App, select 'Server Authentication (with JWT)' authentication method.
+        <br/>If you want to use an existing App, chose one with 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' as Authentication method.

Review Comment:
   ```suggestion
           <br/>If you want to use an existing App, choose one with 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' as Authentication method.
   ```



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r970471528


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/src/main/resources/docs/org.apache.nifi.box.controllerservices.JsonConfigBasedBoxClientService/additionalDetails.html:
##########
@@ -0,0 +1,46 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+
+<h1>Setting up a Box App</h1>
+
+<p>This processor requires a pre-configured App under the Account owning the resources being accessed.</p>
+<p>The App should have the following configuration:</p>
+<ul>
+    <li>
+        If you create a new App, select 'Server Authentication (with JWT)' authentication method.
+        <br/>If you want to use an existing App, chose one with 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' as Authentication method.
+    </li>
+    <li>Should have a 'Client ID' and 'Client Secret'.</li>
+    <li>'App Access Level' should be 'App + Enterprise Access'.</li>
+    <li>'Application Scopes' should have 'Write all files and folders in Box' enabled.</li>
+    <li>'Advanced Features' should have 'Generate user access tokens' and 'Make API calls using the as-user header' enabled.</li>
+    <li>
+        Under 'Add and Manage Public Keys' Generate a Public/Private Keypair and download the configuration JSON file (under App Settings). The full path of this file should be set in the 'Box Config File' property.
+        <br/>Note that you can only download the configuration JSON with the keypair details only once, when you generate the keypair. Also this is the only time Box will show you the private key.
+        <br/>If you want to download the configuration JSON file later (under 'App Settings') - or if you want to use your own keypair - after you download it you need to edit the file and add the keypair details manually.
+    </li>
+    <li>After all settings are done, the App needs to be reauthorized. ('Reauthorize App' on the https://app.box.com/master/custom-apps page.)</li>
+</ul>

Review Comment:
   When the app is configured for the first time, it needs to be authorized which can be requested on the app's page, Authorization tab pressing "Review and Submit" button. 
   The app is not listed on https://app.box.com/master/custom-apps before the first authorization is made. 
   
   I'd keep the note about reauthorization since it's good to know that an authorized app needs to be reauthorized every time an app setting is changed.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services-nar/pom.xml:
##########
@@ -0,0 +1,51 @@
+<?xml version="1.0" encoding="UTF-8"?>

Review Comment:
   Please add LICENSE and NOTICE files to nifi-box-services-nar and nifi-box-api-services-nar.



##########
nifi-assembly/pom.xml:
##########
@@ -850,6 +850,12 @@ language governing permissions and limitations under the License. -->
             <version>1.18.0-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
+        <dependency>

Review Comment:
   Please add the new nars to nifi-assemby.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] turcsanyip commented on pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on PR #6355:
URL: https://github.com/apache/nifi/pull/6355#issuecomment-1244947735

   @tpalfy I managed to set up my app / account properly and to download files with the processors in the end. The user has to authorize the app from the Admin Console. This step should also be mentioned on the Additional Details page, maybe in a separate section after the app setup.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r967200579


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/resources/docs/org.apache.nifi.processors.box.ListBoxFiles/additionalDetails.html:
##########
@@ -0,0 +1,35 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ListBoxFiles</title>
+
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+<p>This processor requires a pre-configured App under the User owning the Folder from which files are being listed.</p>
+<p>The App should have the following configuration:</p>
+<ul>
+    <li>Authentication method should be 'OAuth 2.0 with JSON Web Tokens (Server Authentication)'</li>

Review Comment:
   When a new app is created the authentication method can be selected as "Server Authentication (with JWT)".
   Later on it is displayed in App details as you mentioned  ('OAuth 2.0 with JSON Web Tokens (Server Authentication)'). 
   Maybe it's worth mentioning the other format of authentication method name as well to avoid confusion. 
   



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/BoxTrait.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.Reader;
+
+public interface BoxTrait {
+    PropertyDescriptor USER_ID = new PropertyDescriptor.Builder()

Review Comment:
   The description  of USER_ID ("... owns the listed _folder_")sounds a bit strange in the context of FetchBoxFiles since we have a FILE_ID in that case, not a folder.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r974063888


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/src/main/java/org/apache/nifi/box/controllerservices/JsonConfigBasedBoxClientService.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.box.controllerservices;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.JsonValidator;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+@CapabilityDescription("Provides Box client objects through which Box API calls can be used.")
+@Tags({"box", "client", "provider"})
+public class JsonConfigBasedBoxClientService extends AbstractControllerService implements BoxClientService {
+    public static final PropertyDescriptor ACCOUNT_ID = new PropertyDescriptor.Builder()
+        .name("box-account-id")
+        .displayName("Account ID")
+        .description("The ID of the Box account who owns the accessed resource. Same as 'User Id' under 'App Info' in the App 'General Settings'.")
+        .required(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor APP_CONFIG_FILE = new PropertyDescriptor.Builder()
+        .name("app-config-file")
+        .displayName("App Config File")
+        .description("Full path of an App config JSON file. See Additional Details for more information.")
+        .required(false)
+        .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor APP_CONFIG_JSON = new PropertyDescriptor.Builder()
+        .name("app-config-json")
+        .displayName("App Config JSON")
+        .description("The raw JSON containing an App config. See Additional Details for more information.")
+        .required(false)
+        .sensitive(true)
+        .addValidator(JsonValidator.INSTANCE)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+        ACCOUNT_ID,
+        APP_CONFIG_FILE,
+        APP_CONFIG_JSON,
+        ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+    ));
+
+    private volatile BoxAPIConnection boxAPIConnection;
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        boxAPIConnection = createBoxApiConnection(context);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        Collection<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
+
+        if (validationContext.getProperty(APP_CONFIG_FILE).isSet() && validationContext.getProperty(APP_CONFIG_JSON).isSet()) {
+            validationResults.add(new ValidationResult.Builder()
+                .subject("App configuration")
+                .valid(false)
+                .explanation(String.format("'%s' and '%s' cannot be configured at the same time",
+                    APP_CONFIG_FILE.getDisplayName(),
+                    APP_CONFIG_JSON.getDisplayName())
+                )
+                .build());
+        }
+
+        if (!validationContext.getProperty(APP_CONFIG_FILE).isSet() && !validationContext.getProperty(APP_CONFIG_JSON).isSet()) {
+            validationResults.add(new ValidationResult.Builder()
+                .subject("App configuration")
+                .valid(false)
+                .explanation(String.format("either '%s' or '%s' must be configured",
+                    APP_CONFIG_FILE.getDisplayName(),
+                    APP_CONFIG_JSON.getDisplayName())
+                )
+                .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public BoxAPIConnection getBoxApiConnection() {
+        return boxAPIConnection;
+    }
+
+    private BoxAPIConnection createBoxApiConnection(ConfigurationContext context) {
+        BoxAPIConnection api;
+
+        String accountId = context.getProperty(ACCOUNT_ID).evaluateAttributeExpressions().getValue();
+        ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+
+        BoxConfig boxConfig;
+        if (context.getProperty(APP_CONFIG_FILE).isSet()) {
+            String appConfigFile = context.getProperty(APP_CONFIG_FILE).evaluateAttributeExpressions().getValue();
+            try (
+                Reader reader = new FileReader(appConfigFile);
+            ) {
+                boxConfig = BoxConfig.readFrom(reader);
+            } catch (FileNotFoundException e) {
+                throw new ProcessException("Couldn't find Box config file", e);
+            } catch (IOException e) {
+                throw new ProcessException("Couldn't read Box config file", e);
+            }
+        } else {
+            String appConfig = context.getProperty(APP_CONFIG_JSON).evaluateAttributeExpressions().getValue();
+            boxConfig = BoxConfig.readFrom(appConfig);
+        }
+
+        api = BoxDeveloperEditionAPIConnection.getAppEnterpriseConnection(boxConfig);
+        api.setProxy(proxyConfiguration.createProxy());

Review Comment:
   @krisztina-zsihovszki @tpalfy I re-tested it with the updated code (setting proxy username and password) and it still does not work. I debugged it a bit and the issue seems to be in the Box API itself but it needs to be clarified.
   I suggest removing the proxy property for now and going forward with merging the processors because they are valuable additions even without proxy support. The proxy issue can be investigated further in a follow-up jira.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] tpalfy commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r966201974


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/test/java/org/apache/nifi/processors/box/AbstractBoxFilesIT.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileReader;
+import java.io.Reader;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Set the following constants before running:<br />
+ * <br />
+ * FOLDER_ID - The ID of a Folder the test can use to create files and sub-folders within.<br />
+ * USER_ID - The ID of the user owning the Folder.<br />
+ * BOX_CONFIG_FILE - An App Settings Configuration JSON file. The app needs to have 'OAuth 2.0 with JSON Web Tokens (Server Authentication)' authentication method,
+ * have App + Enterprise access level and be able to read and write files and folders as well as generate user access tokens.<br />
+ * <br />
+ * Created files and folders are cleaned up, but it's advisable to dedicate a folder for this test so that it can be cleaned up easily should the test fail to do so.
+ */
+public abstract class AbstractBoxFilesIT<T extends BoxTrait & Processor> {
+    static final String FOLDER_ID = "";
+    static final String USER_ID = "";
+    static final String BOX_CONFIG_FILE = "";
+
+    protected static final String DEFAULT_FILE_CONTENT = "test_content";
+    public static final String MAIN_FOLDER_NAME = "main";
+
+    protected T testSubject;
+    protected TestRunner testRunner;
+
+    protected BoxAPIConnection boxAPIConnection;
+
+    protected String targetFolderName;
+    protected String mainFolderId;
+
+    protected abstract T createTestSubject();
+
+    @BeforeEach
+    protected void init() throws Exception {
+        testSubject = createTestSubject();
+        testRunner = createTestRunner();
+
+        try (
+            Reader reader = new FileReader(BOX_CONFIG_FILE);
+        ) {
+            BoxConfig boxConfig = BoxConfig.readFrom(reader);
+            boxAPIConnection = BoxDeveloperEditionAPIConnection.getAppEnterpriseConnection(boxConfig);
+            boxAPIConnection.asUser(USER_ID);
+        }
+
+        targetFolderName = new BoxFolder(boxAPIConnection, FOLDER_ID).getInfo("name").getName();
+
+        BoxFolder.Info mainFolderInfo = createFolder(MAIN_FOLDER_NAME, FOLDER_ID);
+        mainFolderId = mainFolderInfo.getID();
+    }
+
+    @AfterEach
+    protected void tearDown() {
+        if (boxAPIConnection != null) {
+            BoxFolder folder = new BoxFolder(boxAPIConnection, mainFolderId);
+            folder.delete(true);
+        }
+    }
+
+    protected TestRunner createTestRunner() {
+        TestRunner testRunner = TestRunners.newTestRunner(testSubject);

Review Comment:
   I think it's better though to have a simple variable instead of an expression as a return value.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r972431295


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services-api/pom.xml:
##########
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>nifi-box-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-box-services-api</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.box</groupId>
+            <artifactId>box-java-sdk</artifactId>
+            <version>3.4.0</version>
+        </dependency>

Review Comment:
   `3.6.0` has been released in the meantime. Please use the latest version.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/pom.xml:
##########
@@ -0,0 +1,51 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>nifi-box-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-box-services</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-box-services-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>

Review Comment:
   It should be `provided` because it comes from the parent NAR.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/ListBoxFiles.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import com.box.sdk.BoxItem;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.box.controllerservices.BoxClientService;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"box", "storage"})
+@CapabilityDescription("Lists files in a Box folder. " +
+    "Each listed file may result in one FlowFile, the metadata being written as FlowFile attributes. " +
+    "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single FlowFile. " +
+    "This Processor is designed to run on Primary Node only in a cluster. If the primary node changes, the new Primary Node will pick up where the " +
+    "previous node left off without duplicating all of the data.")
+@SeeAlso({FetchBoxFiles.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = BoxFileInfo.ID, description = "The id of the file"),
+    @WritesAttribute(attribute = BoxFileInfo.FILENAME, description = "The name of the file"),
+    @WritesAttribute(attribute = BoxFileInfo.PATH, description = "The path of the file on Box"),
+    @WritesAttribute(attribute = BoxFileInfo.SIZE, description = "The size of the file (in bytes)"),
+    @WritesAttribute(attribute = BoxFileInfo.TIMESTAMP, description = "The last modified time of the file.")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "The processor stores necessary data to be able to keep track what files have been listed already." +
+    " What exactly needs to be stored depends on the 'Listing Strategy'.")
+public class ListBoxFiles extends AbstractListProcessor<BoxFileInfo> {
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+        .name("box-folder-id")
+        .displayName("Folder ID")
+        .description("The ID of the folder from which to pull list of files.")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .required(true)
+        .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+        .name("recursive-search")
+        .displayName("Search Recursively")
+        .description("When 'true', will include list of files from sub-folders." +
+            " Otherwise, will return only files that are within the folder defined by the 'Folder ID' property.")
+        .required(true)
+        .defaultValue("true")
+        .allowableValues("true", "false")
+        .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+        .name("min-age")
+        .displayName("Minimum File Age")
+        .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored.")
+        .required(true)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+
+    public static final PropertyDescriptor LISTING_STRATEGY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(AbstractListProcessor.LISTING_STRATEGY)
+        .allowableValues(BY_TIMESTAMPS, BY_ENTITIES, BY_TIME_WINDOW, NO_TRACKING)
+        .build();
+
+    public static final PropertyDescriptor TRACKING_STATE_CACHE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.TRACKING_STATE_CACHE)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    public static final PropertyDescriptor TRACKING_TIME_WINDOW = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.TRACKING_TIME_WINDOW)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    public static final PropertyDescriptor INITIAL_LISTING_TARGET = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.INITIAL_LISTING_TARGET)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+        FOLDER_ID,
+        BoxClientService.BOX_CLIENT_SERVICE,
+        RECURSIVE_SEARCH,

Review Comment:
   Please move the `Box Client Service` to the first position (similar to GoogleDrive and Dropbox processors).
   ```suggestion
           BoxClientService.BOX_CLIENT_SERVICE,
           FOLDER_ID,
           RECURSIVE_SEARCH,
   ```



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-nar/src/main/resources/META-INF/NOTICE:
##########
@@ -0,0 +1,16 @@
+nifi-box-nar
+Copyright 2015-2020 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) The Box SDK for Java
+    The following NOTICE information applies:
+      The Box SDK for Java
+      Copyright 2019 Box, Inc. All rights reserved.

Review Comment:
   Box SDK should not be packaged in the `nifi-box-services-nar`, this entry will be not needed in that case.
   
   However, Jackson jars are present in the NAR (coming with nifi-json-utils) and it needs to be mentioned in the NOTICE file.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/pom.xml:
##########
@@ -0,0 +1,79 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-box-bundle</artifactId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-box-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.box</groupId>
+            <artifactId>box-java-sdk</artifactId>
+            <version>3.4.0</version>
+        </dependency>

Review Comment:
   It comes with `nifi-box-services-api` transitively and it should not be declared here (otherwise it will be packaged into the processors NAR unnecessarily).



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/BoxFileInfo.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import org.apache.nifi.processor.util.list.ListableEntity;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BoxFileInfo implements ListableEntity {
+    public static final String ID = "box.id";
+    public static final String FILENAME = "filename";
+    public static final String PATH = "box.path";

Review Comment:
   Please use `path` instead (the usual `path` + `filename` attribute pair).



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services-api-nar/src/main/resources/META-INF/NOTICE:
##########
@@ -0,0 +1,16 @@
+nifi-box-nar
+Copyright 2015-2020 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) The Box SDK for Java
+    The following NOTICE information applies:
+      The Box SDK for Java
+      Copyright 2019 Box, Inc. All rights reserved.

Review Comment:
   Box SDK has some transitive dependencies that are packaged in the NAR and therefore they should be mentioned in the NOTICE file(s).
   ```
   [INFO] +- com.box:box-java-sdk:jar:3.4.0:compile
   [INFO] |  +- com.eclipsesource.minimal-json:minimal-json:jar:0.9.5:runtime
   [INFO] |  +- org.bitbucket.b_c:jose4j:jar:0.7.9:runtime
   [INFO] |  +- org.bouncycastle:bcprov-jdk15on:jar:1.70:runtime
   [INFO] |  \- org.bouncycastle:bcpkix-jdk15on:jar:1.70:runtime
   [INFO] |     \- org.bouncycastle:bcutil-jdk15on:jar:1.70:runtime
   ```
   



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services-nar/src/main/resources/META-INF/NOTICE:
##########
@@ -0,0 +1,16 @@
+nifi-box-nar
+Copyright 2015-2020 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) The Box SDK for Java
+    The following NOTICE information applies:
+      The Box SDK for Java
+      Copyright 2019 Box, Inc. All rights reserved.

Review Comment:
   Box SDK is not packaged in the `nifi-box-services-nar` so this entry is not needed.
   
   However, Jackson jars are present in the NAR (coming with `nifi-json-utils`) and it needs to be mentioned in the NOTICE file. 



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/pom.xml:
##########
@@ -0,0 +1,79 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-box-bundle</artifactId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-box-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>com.box</groupId>
+            <artifactId>box-java-sdk</artifactId>
+            <version>3.4.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-box-services-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>

Review Comment:
   It should be `provided` because it comes from the parent NAR.



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/ListBoxFiles.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxFile;
+import com.box.sdk.BoxFolder;
+import com.box.sdk.BoxItem;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.box.controllerservices.BoxClientService;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"box", "storage"})
+@CapabilityDescription("Lists files in a Box folder. " +
+    "Each listed file may result in one FlowFile, the metadata being written as FlowFile attributes. " +
+    "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single FlowFile. " +
+    "This Processor is designed to run on Primary Node only in a cluster. If the primary node changes, the new Primary Node will pick up where the " +
+    "previous node left off without duplicating all of the data.")
+@SeeAlso({FetchBoxFiles.class})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = BoxFileInfo.ID, description = "The id of the file"),
+    @WritesAttribute(attribute = BoxFileInfo.FILENAME, description = "The name of the file"),
+    @WritesAttribute(attribute = BoxFileInfo.PATH, description = "The path of the file on Box"),
+    @WritesAttribute(attribute = BoxFileInfo.SIZE, description = "The size of the file (in bytes)"),
+    @WritesAttribute(attribute = BoxFileInfo.TIMESTAMP, description = "The last modified time of the file.")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "The processor stores necessary data to be able to keep track what files have been listed already." +
+    " What exactly needs to be stored depends on the 'Listing Strategy'.")
+public class ListBoxFiles extends AbstractListProcessor<BoxFileInfo> {
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+        .name("box-folder-id")
+        .displayName("Folder ID")
+        .description("The ID of the folder from which to pull list of files.")
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .required(true)
+        .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+        .name("recursive-search")
+        .displayName("Search Recursively")
+        .description("When 'true', will include list of files from sub-folders." +
+            " Otherwise, will return only files that are within the folder defined by the 'Folder ID' property.")
+        .required(true)
+        .defaultValue("true")
+        .allowableValues("true", "false")
+        .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+        .name("min-age")
+        .displayName("Minimum File Age")
+        .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored.")
+        .required(true)
+        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+        .defaultValue("0 sec")
+        .build();
+
+    public static final PropertyDescriptor LISTING_STRATEGY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(AbstractListProcessor.LISTING_STRATEGY)
+        .allowableValues(BY_TIMESTAMPS, BY_ENTITIES, BY_TIME_WINDOW, NO_TRACKING)
+        .build();
+
+    public static final PropertyDescriptor TRACKING_STATE_CACHE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.TRACKING_STATE_CACHE)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    public static final PropertyDescriptor TRACKING_TIME_WINDOW = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.TRACKING_TIME_WINDOW)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    public static final PropertyDescriptor INITIAL_LISTING_TARGET = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(ListedEntityTracker.INITIAL_LISTING_TARGET)
+        .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+        .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+        FOLDER_ID,
+        BoxClientService.BOX_CLIENT_SERVICE,
+        RECURSIVE_SEARCH,
+        MIN_AGE,
+        LISTING_STRATEGY,
+        TRACKING_STATE_CACHE,
+        TRACKING_TIME_WINDOW,
+        INITIAL_LISTING_TARGET,
+        RECORD_WRITER
+    ));
+
+    private volatile BoxAPIConnection boxAPIConnection;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Map<String, String> createAttributes(
+        final BoxFileInfo entity,
+        final ProcessContext context
+    ) {
+        final Map<String, String> attributes = new HashMap<>();
+
+        for (BoxFlowFileAttribute attribute : BoxFlowFileAttribute.values()) {
+            Optional.ofNullable(attribute.getValue(entity))
+                .ifPresent(value -> attributes.put(attribute.getName(), value));
+        }
+
+        return attributes;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws IOException {
+        BoxClientService boxClientService = context.getProperty(BoxClientService.BOX_CLIENT_SERVICE).asControllerService(BoxClientService.class);
+
+        boxAPIConnection = boxClientService.getBoxApiConnection();
+    }
+
+    @Override
+    protected String getListingContainerName(final ProcessContext context) {
+        return String.format("Box Folder [%s]", getPath(context));
+    }
+
+    @Override
+    protected String getPath(final ProcessContext context) {
+        return context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(final PropertyDescriptor property) {
+        return LISTING_STRATEGY.equals(property)
+            || FOLDER_ID.equals(property)
+            || RECURSIVE_SEARCH.equals(property);
+    }
+
+    @Override
+    protected Scope getStateScope(final PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return BoxFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected List<BoxFileInfo> performListing(
+        final ProcessContext context,
+        final Long minTimestamp,
+        final ListingMode listingMode
+    ) throws IOException {
+        final List<BoxFileInfo> listing = new ArrayList<>();
+
+        final String folderId = context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+        final Boolean recursive = context.getProperty(RECURSIVE_SEARCH).asBoolean();
+        final Long minAge = context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
+
+
+        listFolder(listing, folderId, recursive, minAge);
+
+        return listing;
+    }
+
+    private void listFolder(List<BoxFileInfo> listing, String folderId, Boolean recursive, Long minAge) {
+        BoxFolder folder = getFolder(folderId);
+        for (BoxItem.Info itemInfo : folder.getChildren(
+            "id",
+            "name",
+            "item_status",
+            "size",
+            "created_at",
+            "modified_at",
+            "content_created_at",
+            "content_modified_at",
+            "path_collection"
+        )) {
+            if (itemInfo instanceof BoxFile.Info) {
+                BoxFile.Info info = (BoxFile.Info) itemInfo;
+
+                long createdAt = itemInfo.getCreatedAt().getTime();
+
+                if (Instant.now().toEpochMilli() - createdAt >= minAge) {
+                    BoxFileInfo boxFileInfo = new BoxFileInfo.Builder()
+                        .id(info.getID())
+                        .fileName(info.getName())
+                        .path("/" + info.getPathCollection().stream()
+                            .filter(pathItemInfo -> !pathItemInfo.getID().equals("0"))
+                            .map(BoxItem.Info::getName)
+                            .collect(Collectors.joining("/")))
+                        .size(info.getSize())
+                        .createdTime(info.getCreatedAt().getTime())
+                        .modifiedTime(info.getModifiedAt().getTime())
+                        .build();
+
+                    listing.add(boxFileInfo);
+                }
+            } else if (recursive && itemInfo instanceof BoxFolder.Info) {
+                BoxFolder.Info info = (BoxFolder.Info) itemInfo;
+                listFolder(listing, info.getID(), recursive, minAge);
+            }
+        }
+    }

Review Comment:
   `Instant.now()` should be calculated once and before all network communications. Otherwise newer and newer files can be listed breaking the purpose of `minAge`.
   As the method is recursive, the value should come from outside as a parameter. I'd suggest passing in `createdAtMax` instead of `minAge` which can be calculated as `Instant.now() - minAge`.
   ```suggestion
           long createdAtMax = Instant.now().toEpochMilli() - minAge;
           listFolder(listing, folderId, recursive, createdAtMax);
   
           return listing;
       }
   
       private void listFolder(List<BoxFileInfo> listing, String folderId, Boolean recursive, long createdAtMax) {
           BoxFolder folder = getFolder(folderId);
           for (BoxItem.Info itemInfo : folder.getChildren(
               "id",
               "name",
               "item_status",
               "size",
               "created_at",
               "modified_at",
               "content_created_at",
               "content_modified_at",
               "path_collection"
           )) {
               if (itemInfo instanceof BoxFile.Info) {
                   BoxFile.Info info = (BoxFile.Info) itemInfo;
   
                   long createdAt = itemInfo.getCreatedAt().getTime();
   
                   if (createdAt <= createdAtMax) {
                       BoxFileInfo boxFileInfo = new BoxFileInfo.Builder()
                           .id(info.getID())
                           .fileName(info.getName())
                           .path("/" + info.getPathCollection().stream()
                               .filter(pathItemInfo -> !pathItemInfo.getID().equals("0"))
                               .map(BoxItem.Info::getName)
                               .collect(Collectors.joining("/")))
                           .size(info.getSize())
                           .createdTime(info.getCreatedAt().getTime())
                           .modifiedTime(info.getModifiedAt().getTime())
                           .build();
   
                       listing.add(boxFileInfo);
                   }
               } else if (recursive && itemInfo instanceof BoxFolder.Info) {
                   BoxFolder.Info info = (BoxFolder.Info) itemInfo;
                   listFolder(listing, info.getID(), recursive, createdAtMax);
               }
           }
       }
   ```



##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-processors/src/main/java/org/apache/nifi/processors/box/FetchBoxFiles.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.box;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxAPIResponseException;
+import com.box.sdk.BoxFile;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.box.controllerservices.BoxClientService;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"box", "storage", "fetch"})
+@CapabilityDescription("Fetches files from a Box Folder. Designed to be used in tandem with ListBoxFiles.")
+@SeeAlso({ListBoxFiles.class})
+@WritesAttributes({
+    @WritesAttribute(attribute = FetchBoxFiles.ERROR_CODE_ATTRIBUTE, description = "The error code returned by Box when the fetch of a file fails"),
+    @WritesAttribute(attribute = FetchBoxFiles.ERROR_MESSAGE_ATTRIBUTE, description = "The error message returned by Box when the fetch of a file fails")
+})
+public class FetchBoxFiles extends AbstractProcessor {
+    public static final String ERROR_CODE_ATTRIBUTE = "error.code";
+    public static final String ERROR_MESSAGE_ATTRIBUTE = "error.message";
+
+    public static final PropertyDescriptor FILE_ID = new PropertyDescriptor
+        .Builder().name("box-file-id")
+        .displayName("File ID")
+        .description("The ID of the File to fetch")
+        .required(true)
+        .defaultValue("${box.id}")
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
+    public static final Relationship REL_SUCCESS =
+        new Relationship.Builder()
+            .name("success")
+            .description("A FlowFile will be routed here for each successfully fetched File.")
+            .build();
+
+    public static final Relationship REL_FAILURE =
+        new Relationship.Builder().name("failure")
+            .description("A FlowFile will be routed here for each File for which fetch was attempted but failed.")
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+        FILE_ID,
+        BoxClientService.BOX_CLIENT_SERVICE

Review Comment:
   Please move the Box Client Service to the first position (similar to GoogleDrive and Dropbox processors).



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
krisztina-zsihovszki commented on code in PR #6355:
URL: https://github.com/apache/nifi/pull/6355#discussion_r972708814


##########
nifi-nar-bundles/nifi-box-bundle/nifi-box-services/src/main/java/org/apache/nifi/box/controllerservices/JsonConfigBasedBoxClientService.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.box.controllerservices;
+
+import com.box.sdk.BoxAPIConnection;
+import com.box.sdk.BoxConfig;
+import com.box.sdk.BoxDeveloperEditionAPIConnection;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.JsonValidator;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxySpec;
+
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+@CapabilityDescription("Provides Box client objects through which Box API calls can be used.")
+@Tags({"box", "client", "provider"})
+public class JsonConfigBasedBoxClientService extends AbstractControllerService implements BoxClientService {
+    public static final PropertyDescriptor ACCOUNT_ID = new PropertyDescriptor.Builder()
+        .name("box-account-id")
+        .displayName("Account ID")
+        .description("The ID of the Box account who owns the accessed resource. Same as 'User Id' under 'App Info' in the App 'General Settings'.")
+        .required(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor APP_CONFIG_FILE = new PropertyDescriptor.Builder()
+        .name("app-config-file")
+        .displayName("App Config File")
+        .description("Full path of an App config JSON file. See Additional Details for more information.")
+        .required(false)
+        .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor APP_CONFIG_JSON = new PropertyDescriptor.Builder()
+        .name("app-config-json")
+        .displayName("App Config JSON")
+        .description("The raw JSON containing an App config. See Additional Details for more information.")
+        .required(false)
+        .sensitive(true)
+        .addValidator(JsonValidator.INSTANCE)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+        ACCOUNT_ID,
+        APP_CONFIG_FILE,
+        APP_CONFIG_JSON,
+        ProxyConfiguration.createProxyConfigPropertyDescriptor(false, PROXY_SPECS)
+    ));
+
+    private volatile BoxAPIConnection boxAPIConnection;
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        boxAPIConnection = createBoxApiConnection(context);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        Collection<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
+
+        if (validationContext.getProperty(APP_CONFIG_FILE).isSet() && validationContext.getProperty(APP_CONFIG_JSON).isSet()) {
+            validationResults.add(new ValidationResult.Builder()
+                .subject("App configuration")
+                .valid(false)
+                .explanation(String.format("'%s' and '%s' cannot be configured at the same time",
+                    APP_CONFIG_FILE.getDisplayName(),
+                    APP_CONFIG_JSON.getDisplayName())
+                )
+                .build());
+        }
+
+        if (!validationContext.getProperty(APP_CONFIG_FILE).isSet() && !validationContext.getProperty(APP_CONFIG_JSON).isSet()) {
+            validationResults.add(new ValidationResult.Builder()
+                .subject("App configuration")
+                .valid(false)
+                .explanation(String.format("either '%s' or '%s' must be configured",
+                    APP_CONFIG_FILE.getDisplayName(),
+                    APP_CONFIG_JSON.getDisplayName())
+                )
+                .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public BoxAPIConnection getBoxApiConnection() {
+        return boxAPIConnection;
+    }
+
+    private BoxAPIConnection createBoxApiConnection(ConfigurationContext context) {
+        BoxAPIConnection api;
+
+        String accountId = context.getProperty(ACCOUNT_ID).evaluateAttributeExpressions().getValue();
+        ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+
+        BoxConfig boxConfig;
+        if (context.getProperty(APP_CONFIG_FILE).isSet()) {
+            String appConfigFile = context.getProperty(APP_CONFIG_FILE).evaluateAttributeExpressions().getValue();
+            try (
+                Reader reader = new FileReader(appConfigFile);
+            ) {
+                boxConfig = BoxConfig.readFrom(reader);
+            } catch (FileNotFoundException e) {
+                throw new ProcessException("Couldn't find Box config file", e);
+            } catch (IOException e) {
+                throw new ProcessException("Couldn't read Box config file", e);
+            }
+        } else {
+            String appConfig = context.getProperty(APP_CONFIG_JSON).evaluateAttributeExpressions().getValue();
+            boxConfig = BoxConfig.readFrom(appConfig);
+        }
+
+        api = BoxDeveloperEditionAPIConnection.getAppEnterpriseConnection(boxConfig);
+        api.setProxy(proxyConfiguration.createProxy());

Review Comment:
   The Proxy object which is set for the API does not contain username/password, when I tested the processor with username/password auth in the proxy, I got "HTTP/1.1 407 Proxy Authentication Required".
   The username, password need to be passed to the Box API as well.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi] asfgit closed pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #6355: NIFI-10427 - Added ListBoxFiles and FetchBoxFiles processor.
URL: https://github.com/apache/nifi/pull/6355


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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