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/12/01 10:51:20 UTC

[GitHub] [nifi] krisztina-zsihovszki opened a new pull request, #6740: NIFI-10868 Add PutDropbox processor

krisztina-zsihovszki opened a new pull request, #6740:
URL: https://github.com/apache/nifi/pull/6740

   <!-- 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-10868](https://issues.apache.org/jira/browse/NIFI-10868)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-10868`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-10868`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [x] 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`
     - [x] JDK 8
     - [x] JDK 11
     - [x] 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
   
   - [x] 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 a diff in pull request #6740: NIFI-10868 Add PutDropbox processor

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


##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.RateLimitException;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.UploadErrorException;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int SINGLE_UPLOAD_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final String IGNORE_RESOLUTION = "ignore";
+    public static final String OVERWRITE_RESOLUTION = "overwrite";
+    public static final String FAIL_RESOLUTION = "fail";
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The path of the Dropbox folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("Filename")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder()
+            .name("conflict-resolution-strategy")
+            .displayName("Conflict Resolution Strategy")
+            .description("Indicates what should happen when a file with the same name already exists in the specified Dropbox folder.")
+            .required(true)
+            .defaultValue(FAIL_RESOLUTION)
+            .allowableValues(FAIL_RESOLUTION, IGNORE_RESOLUTION, OVERWRITE_RESOLUTION)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_SIZE = new PropertyDescriptor.Builder()
+            .name("chunked-upload-size")
+            .displayName("Chunked Upload Size")
+            .description("Defines the size of a chunk. Used when a FlowFile's size exceeds 'Chunked Upload Threshold' and content is uploaded in smaller chunks. "
+                    + "It is recommended to specify chunked upload size smaller than 'Chunked Upload Threshold' and as multiples of 4 MB. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("8 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_THRESHOLD = new PropertyDescriptor.Builder()
+            .name("chunked-upload-threshold")
+            .displayName("Chunked Upload Threshold")
+            .description("The maximum size of the content which is uploaded at once. FlowFiles larger than this threshold are uploaded in chunks. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("150 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            CONFLICT_RESOLUTION,
+            CHUNKED_UPLOAD_THRESHOLD,
+            CHUNKED_UPLOAD_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+
+    private DbxUploader<?, ?, ?> dbxUploader;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        dropboxApiClient = getDropboxApiClient(context, getIdentifier());
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final String folder = context.getProperty(FOLDER).evaluateAttributeExpressions(flowFile).getValue();
+        final String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        final long chunkUploadThreshold = context.getProperty(CHUNKED_UPLOAD_THRESHOLD)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final long uploadChunkSize = context.getProperty(CHUNKED_UPLOAD_SIZE)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final String conflictResolution = context.getProperty(CONFLICT_RESOLUTION).getValue();
+
+        boolean uploadErrorOccurred = false;
+
+        final long size = flowFile.getSize();
+        final String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        try (final InputStream rawIn = session.read(flowFile)) {
+            try {
+                if (size <= chunkUploadThreshold) {
+                    try (UploadUploader uploader = createUploadUploader(uploadPath, conflictResolution)) {
+                        uploader.uploadAndFinish(rawIn);
+                    }
+                } else {
+                    uploadLargeFileInChunks(uploadPath, rawIn, size, uploadChunkSize, conflictResolution);
+                }
+            } catch (UploadErrorException e) {
+                handleUploadError(conflictResolution, uploadPath, e);
+            } catch (RateLimitException e) {
+                context.yield();
+                throw new ProcessException("Dropbox API rate limit exceeded while uploading file", e);
+            }
+        } catch (Exception e) {
+            getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+            uploadErrorOccurred = true;
+        } finally {
+            dbxUploader.close();
+        }
+
+        if (!uploadErrorOccurred) {
+            session.transfer(flowFile, REL_SUCCESS);
+        } else {
+            session.transfer(flowFile, REL_FAILURE);
+        }
+    }
+
+    @OnUnscheduled
+    public void shutdown() {
+        if (dbxUploader != null) {
+            dbxUploader.close();
+        }
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    private void handleUploadError(final String conflictResolution, final String uploadPath, final UploadErrorException e) throws UploadErrorException {
+        if (e.errorValue.isPath() && e.errorValue.getPathValue().getReason().isConflict()) {
+
+            if (IGNORE_RESOLUTION.equals(conflictResolution)) {
+                getLogger().info("File with the same name [{}] already exists. Remote file is not modified due to {} being set to '{}'.",
+                        uploadPath, CONFLICT_RESOLUTION.getDisplayName(), conflictResolution);
+                return;
+            } else if (conflictResolution.equals(FAIL_RESOLUTION)) {
+                throw new ProcessException(format("File with the same name [%s] already exists.", uploadPath), e);
+            }
+        }
+        throw new ProcessException(e);
+    }
+
+    private void uploadLargeFileInChunks(String path, InputStream rawIn, long size, long uploadChunkSize,  String conflictResolution) throws Exception {

Review Comment:
   Minor:
   ```suggestion
       private void uploadLargeFileInChunks(String path, InputStream rawIn, long size, long uploadChunkSize,  String conflictResolution) throws DbxException, IOException {
   ```



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.RateLimitException;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.UploadErrorException;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int SINGLE_UPLOAD_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final String IGNORE_RESOLUTION = "ignore";
+    public static final String OVERWRITE_RESOLUTION = "overwrite";
+    public static final String FAIL_RESOLUTION = "fail";
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The path of the Dropbox folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("Filename")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder()
+            .name("conflict-resolution-strategy")
+            .displayName("Conflict Resolution Strategy")
+            .description("Indicates what should happen when a file with the same name already exists in the specified Dropbox folder.")
+            .required(true)
+            .defaultValue(FAIL_RESOLUTION)
+            .allowableValues(FAIL_RESOLUTION, IGNORE_RESOLUTION, OVERWRITE_RESOLUTION)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_SIZE = new PropertyDescriptor.Builder()
+            .name("chunked-upload-size")
+            .displayName("Chunked Upload Size")
+            .description("Defines the size of a chunk. Used when a FlowFile's size exceeds 'Chunked Upload Threshold' and content is uploaded in smaller chunks. "
+                    + "It is recommended to specify chunked upload size smaller than 'Chunked Upload Threshold' and as multiples of 4 MB. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("8 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_THRESHOLD = new PropertyDescriptor.Builder()
+            .name("chunked-upload-threshold")
+            .displayName("Chunked Upload Threshold")
+            .description("The maximum size of the content which is uploaded at once. FlowFiles larger than this threshold are uploaded in chunks. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("150 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            CONFLICT_RESOLUTION,
+            CHUNKED_UPLOAD_THRESHOLD,
+            CHUNKED_UPLOAD_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+
+    private DbxUploader<?, ?, ?> dbxUploader;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        dropboxApiClient = getDropboxApiClient(context, getIdentifier());
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final String folder = context.getProperty(FOLDER).evaluateAttributeExpressions(flowFile).getValue();
+        final String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        final long chunkUploadThreshold = context.getProperty(CHUNKED_UPLOAD_THRESHOLD)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final long uploadChunkSize = context.getProperty(CHUNKED_UPLOAD_SIZE)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final String conflictResolution = context.getProperty(CONFLICT_RESOLUTION).getValue();
+
+        boolean uploadErrorOccurred = false;
+
+        final long size = flowFile.getSize();
+        final String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        try (final InputStream rawIn = session.read(flowFile)) {
+            try {
+                if (size <= chunkUploadThreshold) {
+                    try (UploadUploader uploader = createUploadUploader(uploadPath, conflictResolution)) {
+                        uploader.uploadAndFinish(rawIn);
+                    }
+                } else {
+                    uploadLargeFileInChunks(uploadPath, rawIn, size, uploadChunkSize, conflictResolution);
+                }
+            } catch (UploadErrorException e) {
+                handleUploadError(conflictResolution, uploadPath, e);
+            } catch (RateLimitException e) {
+                context.yield();
+                throw new ProcessException("Dropbox API rate limit exceeded while uploading file", e);
+            }
+        } catch (Exception e) {
+            getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+            uploadErrorOccurred = true;
+        } finally {
+            dbxUploader.close();
+        }
+
+        if (!uploadErrorOccurred) {
+            session.transfer(flowFile, REL_SUCCESS);
+        } else {
+            session.transfer(flowFile, REL_FAILURE);
+        }

Review Comment:
   I would be a clearer code design without the `uploadErrorOccurred` flag. The success case can be moved into the `try` block and the failure into `catch`.
   
   Please also send a provenance event for success and penalize the FlowFile in case of failure.
   Example can be found in [PutAzureDataLakeStorage](https://github.com/apache/nifi/blob/ed6ba537249ea781402dc0ac7e341907ba5b0c94/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/PutAzureDataLakeStorage.java#L144-L157).
   Please note, provenance event is only needed when the upload has really happened (so not when the file exists and conflict strategy is ignore).



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.RateLimitException;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.UploadErrorException;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int SINGLE_UPLOAD_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final String IGNORE_RESOLUTION = "ignore";
+    public static final String OVERWRITE_RESOLUTION = "overwrite";
+    public static final String FAIL_RESOLUTION = "fail";
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The path of the Dropbox folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("Filename")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder()
+            .name("conflict-resolution-strategy")
+            .displayName("Conflict Resolution Strategy")
+            .description("Indicates what should happen when a file with the same name already exists in the specified Dropbox folder.")
+            .required(true)
+            .defaultValue(FAIL_RESOLUTION)
+            .allowableValues(FAIL_RESOLUTION, IGNORE_RESOLUTION, OVERWRITE_RESOLUTION)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_SIZE = new PropertyDescriptor.Builder()
+            .name("chunked-upload-size")
+            .displayName("Chunked Upload Size")
+            .description("Defines the size of a chunk. Used when a FlowFile's size exceeds 'Chunked Upload Threshold' and content is uploaded in smaller chunks. "
+                    + "It is recommended to specify chunked upload size smaller than 'Chunked Upload Threshold' and as multiples of 4 MB. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("8 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_THRESHOLD = new PropertyDescriptor.Builder()
+            .name("chunked-upload-threshold")
+            .displayName("Chunked Upload Threshold")
+            .description("The maximum size of the content which is uploaded at once. FlowFiles larger than this threshold are uploaded in chunks. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("150 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            CONFLICT_RESOLUTION,
+            CHUNKED_UPLOAD_THRESHOLD,
+            CHUNKED_UPLOAD_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+
+    private DbxUploader<?, ?, ?> dbxUploader;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        dropboxApiClient = getDropboxApiClient(context, getIdentifier());
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final String folder = context.getProperty(FOLDER).evaluateAttributeExpressions(flowFile).getValue();
+        final String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        final long chunkUploadThreshold = context.getProperty(CHUNKED_UPLOAD_THRESHOLD)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final long uploadChunkSize = context.getProperty(CHUNKED_UPLOAD_SIZE)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final String conflictResolution = context.getProperty(CONFLICT_RESOLUTION).getValue();
+
+        boolean uploadErrorOccurred = false;
+
+        final long size = flowFile.getSize();
+        final String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        try (final InputStream rawIn = session.read(flowFile)) {
+            try {
+                if (size <= chunkUploadThreshold) {
+                    try (UploadUploader uploader = createUploadUploader(uploadPath, conflictResolution)) {
+                        uploader.uploadAndFinish(rawIn);
+                    }
+                } else {
+                    uploadLargeFileInChunks(uploadPath, rawIn, size, uploadChunkSize, conflictResolution);
+                }
+            } catch (UploadErrorException e) {
+                handleUploadError(conflictResolution, uploadPath, e);
+            } catch (RateLimitException e) {
+                context.yield();
+                throw new ProcessException("Dropbox API rate limit exceeded while uploading file", e);
+            }
+        } catch (Exception e) {
+            getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+            uploadErrorOccurred = true;
+        } finally {
+            dbxUploader.close();

Review Comment:
   `dbxUploader` can be `null` at this point (eg. when connect error occurs and no `DbxUploader` initialized yet) which would lead to an additional NPE beyond the original exception. All `DbxUploader`s are in TWR blocks now so this `close()` call is not necessary anymore and can simply be removed.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.RateLimitException;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.UploadErrorException;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int SINGLE_UPLOAD_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final String IGNORE_RESOLUTION = "ignore";
+    public static final String OVERWRITE_RESOLUTION = "overwrite";
+    public static final String FAIL_RESOLUTION = "fail";
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The path of the Dropbox folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("Filename")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor CONFLICT_RESOLUTION = new PropertyDescriptor.Builder()
+            .name("conflict-resolution-strategy")
+            .displayName("Conflict Resolution Strategy")
+            .description("Indicates what should happen when a file with the same name already exists in the specified Dropbox folder.")
+            .required(true)
+            .defaultValue(FAIL_RESOLUTION)
+            .allowableValues(FAIL_RESOLUTION, IGNORE_RESOLUTION, OVERWRITE_RESOLUTION)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_SIZE = new PropertyDescriptor.Builder()
+            .name("chunked-upload-size")
+            .displayName("Chunked Upload Size")
+            .description("Defines the size of a chunk. Used when a FlowFile's size exceeds 'Chunked Upload Threshold' and content is uploaded in smaller chunks. "
+                    + "It is recommended to specify chunked upload size smaller than 'Chunked Upload Threshold' and as multiples of 4 MB. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("8 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor CHUNKED_UPLOAD_THRESHOLD = new PropertyDescriptor.Builder()
+            .name("chunked-upload-threshold")
+            .displayName("Chunked Upload Threshold")
+            .description("The maximum size of the content which is uploaded at once. FlowFiles larger than this threshold are uploaded in chunks. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("150 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            CONFLICT_RESOLUTION,
+            CHUNKED_UPLOAD_THRESHOLD,
+            CHUNKED_UPLOAD_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+
+    private DbxUploader<?, ?, ?> dbxUploader;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        dropboxApiClient = getDropboxApiClient(context, getIdentifier());
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final String folder = context.getProperty(FOLDER).evaluateAttributeExpressions(flowFile).getValue();
+        final String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        final long chunkUploadThreshold = context.getProperty(CHUNKED_UPLOAD_THRESHOLD)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final long uploadChunkSize = context.getProperty(CHUNKED_UPLOAD_SIZE)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        final String conflictResolution = context.getProperty(CONFLICT_RESOLUTION).getValue();
+
+        boolean uploadErrorOccurred = false;
+
+        final long size = flowFile.getSize();
+        final String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        try (final InputStream rawIn = session.read(flowFile)) {
+            try {
+                if (size <= chunkUploadThreshold) {
+                    try (UploadUploader uploader = createUploadUploader(uploadPath, conflictResolution)) {
+                        uploader.uploadAndFinish(rawIn);
+                    }
+                } else {
+                    uploadLargeFileInChunks(uploadPath, rawIn, size, uploadChunkSize, conflictResolution);
+                }
+            } catch (UploadErrorException e) {
+                handleUploadError(conflictResolution, uploadPath, e);
+            } catch (RateLimitException e) {
+                context.yield();
+                throw new ProcessException("Dropbox API rate limit exceeded while uploading file", e);
+            }
+        } catch (Exception e) {
+            getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+            uploadErrorOccurred = true;
+        } finally {
+            dbxUploader.close();
+        }
+
+        if (!uploadErrorOccurred) {
+            session.transfer(flowFile, REL_SUCCESS);
+        } else {
+            session.transfer(flowFile, REL_FAILURE);
+        }
+    }
+
+    @OnUnscheduled
+    public void shutdown() {
+        if (dbxUploader != null) {
+            dbxUploader.close();
+        }
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }

Review Comment:
   Minor: could you please move it up near to `getRelationships()`.



-- 
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 #6740: NIFI-10868 Add PutDropbox processor

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #6740: NIFI-10868 Add PutDropbox processor
URL: https://github.com/apache/nifi/pull/6740


-- 
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 #6740: NIFI-10868 Add PutDropbox processor

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


##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/test/java/org/apache/nifi/processors/dropbox/PutDropboxTest.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.dropbox;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.DbxUserFilesRequests;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartResult;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.nifi.dropbox.credentials.service.DropboxCredentialService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class PutDropboxTest {
+
+    public static final String TEST_FOLDER = "/testFolder";
+    public static final String FILENAME_1 = "file_name_1";
+    public static final String FILENAME_2 = "file_name_2";
+    private static final String CONTENT = "1234567890";
+    private static final String LARGE_CONTENT_30B = "123456789012345678901234567890";
+    private static final String SESSION_ID = "sessionId";
+    public static final long CHUNK_SIZE_IN_BYTES = 8;
+    public static final long MAX_FILE_SIZE_IN_BYTES = 15;
+    private TestRunner testRunner;
+
+    @Mock
+    private DbxClientV2 mockDropboxClient;
+
+    @Mock
+    private DropboxCredentialService credentialService;

Review Comment:
   Other mock objects have "mock-" prefix in their name.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)

Review Comment:
   I think we should support EL with FlowFile attributes for the Folder (similar to Filename).



-- 
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 #6740: NIFI-10868 Add PutDropbox processor

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


##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.dropbox;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int SINGLE_UPLOAD_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The path of the Dropbox folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("Filename")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("Defines the size of a chunk. Used when a FlowFile's size exceeds CHUNK_UPLOAD_LIMIT and content is uploaded in smaller chunks. "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor CHUNK_UPLOAD_LIMIT = new PropertyDescriptor.Builder()
+            .name("chunk-upload-limit")
+            .displayName("Chunk Upload Limit")
+            .description("The maximum size of the content which is uploaded at once. FlowFiles larger than this limit are uploaded in chunks. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("150 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            CHUNK_UPLOAD_LIMIT,

Review Comment:
   Thanks for adding a property for the limit. I'd suggest the following names and order of these two chunked upload related properties:
   ```
   Chunked Upload Threshold
   Chunked Upload Size
   ```



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/FetchDropbox.java:
##########
@@ -51,7 +49,7 @@
 @CapabilityDescription("Fetches files from Dropbox. Designed to be used in tandem with ListDropbox.")
 @WritesAttribute(attribute = "error.message", description = "When a FlowFile is routed to 'failure', this attribute is added indicating why the file could "
         + "not be fetched from Dropbox.")
-@SeeAlso(ListDropbox.class)
+@SeeAlso({PutDropbox.class, ListDropbox.class})
 @WritesAttributes(
         @WritesAttribute(attribute = FetchDropbox.ERROR_MESSAGE_ATTRIBUTE, description = "The error message returned by Dropbox when the fetch of a file fails."))

Review Comment:
   Just spotted that `@WritesAttribute` is duplicated with the same attribute name (`error.message`).



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.dropbox;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int SINGLE_UPLOAD_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The path of the Dropbox folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("Filename")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("Defines the size of a chunk. Used when a FlowFile's size exceeds CHUNK_UPLOAD_LIMIT and content is uploaded in smaller chunks. "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor CHUNK_UPLOAD_LIMIT = new PropertyDescriptor.Builder()
+            .name("chunk-upload-limit")
+            .displayName("Chunk Upload Limit")
+            .description("The maximum size of the content which is uploaded at once. FlowFiles larger than this limit are uploaded in chunks. "
+                    + "Maximum allowed value is 150 MB.")
+            .defaultValue("150 MB")
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, SINGLE_UPLOAD_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            CHUNK_UPLOAD_LIMIT,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+
+    private static final Set<Relationship> RELATIONSHIPS;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        RELATIONSHIPS = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+
+    private DbxUploader dbxUploader;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        dropboxApiClient = getDropboxApiClient(context, getIdentifier());
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final String folder = context.getProperty(FOLDER).evaluateAttributeExpressions(flowFile).getValue();
+        final String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+        final long uploadFileSizeLimit = context.getProperty(CHUNK_UPLOAD_LIMIT)
+                .asDataSize(DataUnit.B)
+                .longValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        final String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        try (final InputStream rawIn = session.read(flowFile)) {
+            if (size <= uploadFileSizeLimit) {
+                uploadedFileMetadata = createUploadUploader(uploadPath).uploadAndFinish(rawIn);
+            } else {
+                long chunkSize = context.getProperty(UPLOAD_CHUNK_SIZE)
+                        .asDataSize(DataUnit.B)
+                        .longValue();
+
+                uploadedFileMetadata = uploadLargeFileInChunks(rawIn, size, chunkSize, uploadPath);
+            }
+        } catch (Exception e) {
+            getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+        } finally {
+            dbxUploader.close();

Review Comment:
   It is not enough to close `dbxUploader` here because it is reassigned a couple of times in the called methods (like `uploadLargeFileInChunks`). Instead, the local `DbxUploader` objects need to be closed individually (and this global close is not needed).
   The best option is to use TWR blocks, like this:
   ```
           final String sessionId;
           try (UploadSessionStartUploader uploader = createUploadSessionStartUploader()) {
               sessionId = uploader.uploadAndFinish(rawIn, uploadChunkSize).getSessionId();
           }
   ```



-- 
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 #6740: NIFI-10868 Add PutDropbox processor

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


##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
+        String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        if (size > 0) {
+            try (final InputStream rawIn = session.read(flowFile)) {
+                if (size <= getUploadFileSizeLimit()) {
+                    dbxUploader = dropboxApiClient.files()
+                            .upload(uploadPath);
+                    uploadedFileMetadata = ((UploadUploader) dbxUploader).uploadAndFinish(rawIn);
+                } else {
+                    long chunkSize = context.getProperty(UPLOAD_CHUNK_SIZE)
+                            .evaluateAttributeExpressions()
+                            .asDataSize(DataUnit.B)
+                            .longValue();
+
+                    uploadedFileMetadata = uploadLargeFileInChunks(rawIn, size, chunkSize, uploadPath);
+                }
+            } catch (Exception e) {
+                getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+            }
+
+            if (uploadedFileMetadata != null) {
+                session.transfer(flowFile, REL_SUCCESS);
+            } else {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+        }
+    }
+
+    @OnUnscheduled
+    @OnDisabled
+    public synchronized void shutdown() {
+        if (dbxUploader != null) {
+            dbxUploader.close();
+        }
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    protected long getUploadFileSizeLimit() {
+        return UPLOAD_FILE_SIZE_LIMIT_IN_BYTES;
+    }

Review Comment:
   It should be `private` and `static`.
   Is this method needed at all? The validator uses the constant directly. Others can do the same.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
+        String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        if (size > 0) {
+            try (final InputStream rawIn = session.read(flowFile)) {
+                if (size <= getUploadFileSizeLimit()) {
+                    dbxUploader = dropboxApiClient.files()
+                            .upload(uploadPath);
+                    uploadedFileMetadata = ((UploadUploader) dbxUploader).uploadAndFinish(rawIn);
+                } else {
+                    long chunkSize = context.getProperty(UPLOAD_CHUNK_SIZE)
+                            .evaluateAttributeExpressions()
+                            .asDataSize(DataUnit.B)
+                            .longValue();
+
+                    uploadedFileMetadata = uploadLargeFileInChunks(rawIn, size, chunkSize, uploadPath);
+                }
+            } catch (Exception e) {
+                getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+            }
+
+            if (uploadedFileMetadata != null) {
+                session.transfer(flowFile, REL_SUCCESS);
+            } else {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+        }
+    }
+
+    @OnUnscheduled
+    @OnDisabled
+    public synchronized void shutdown() {

Review Comment:
   `@OnDisabled` is not necessary because only an already stopped (and therefore unscheduled) processor can be set disabled.
   `synchronized` is not necessary because the framework calls `@OnUnscheduled` in a single thread per processor.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }

Review Comment:
   Not sure if it can be defined and annotated as `@OnScheduled` in `DropboxTrait` but it is a duplicated method in List, Fetch and Put. Could you please check if it works when the method is in the parent interface?



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
+        String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        if (size > 0) {
+            try (final InputStream rawIn = session.read(flowFile)) {
+                if (size <= getUploadFileSizeLimit()) {
+                    dbxUploader = dropboxApiClient.files()
+                            .upload(uploadPath);
+                    uploadedFileMetadata = ((UploadUploader) dbxUploader).uploadAndFinish(rawIn);

Review Comment:
   I would use type-specific local variables for `UploadUploader`, etc. and assign `dbxUploader` separately. The type cast could be avoided that way and also the explicit `dbxUploader` assignment would make it more clear that it needs to be stored for the `shutdown()` method.
   For example:
   ```
                       UploadUploader uploader = dropboxApiClient.files()
                               .upload(uploadPath);
                       dbxUploader = uploader;
                       uploadedFileMetadata = uploader.uploadAndFinish(rawIn);
   ```
   or creating a `createUploadUploader()` method:
   ```
                   private UploadUploader createUploadUploader(final String uploadPath) {
                       UploadUploader uploader = dropboxApiClient.files()
                               .upload(uploadPath);
                       dbxUploader = uploader;
                       return uploader;
                   }
   ```
   



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();

Review Comment:
   Please use `final`-s where possible.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;

Review Comment:
   ```suggestion
       private static final Set<Relationship> RELATIONSHIPS;
   ```



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
+        String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        if (size > 0) {
+            try (final InputStream rawIn = session.read(flowFile)) {
+                if (size <= getUploadFileSizeLimit()) {

Review Comment:
   We may consider to allow chunked uploading for smaller files as well (based on a property).



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;

Review Comment:
   `UPLOAD_FILE_SIZE_LIMIT` name is a bit misleading because it is possible to upload bigger files. It is rather the limit of a chunk / single upload. I would rather call it `SINGLE_UPLOAD_LIMIT_IN_BYTES` or similar.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
+        String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        if (size > 0) {

Review Comment:
   We should do something with the 0-byte input as well. Otherwise we get an error because the FF is not transferred and in general processor should not swallow any FFs but send it to a relationship.
   Does the dropbox api support uploading empty files? Other put processors (maybe not all) upload those files too.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
+        String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        if (size > 0) {
+            try (final InputStream rawIn = session.read(flowFile)) {
+                if (size <= getUploadFileSizeLimit()) {
+                    dbxUploader = dropboxApiClient.files()
+                            .upload(uploadPath);

Review Comment:
   `DbxUploader` is `AutoCloseable`. Should not we close it after usage?



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")

Review Comment:
   I believe `Filename` is more common.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))

Review Comment:
   The property supports variables but `createDataSizeBoundsValidator` does not check those values.
   For this reason, the evaluated `chunkSize` would need to be validated in `onTrigger()`.
   Or a more generic approach would be to extend the validator to check the value when EL is present but only variables are supported (similar to `createRegexMatchingValidator()` used for the `FOLDER` property).



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})

Review Comment:
   Please add cross-references to this processor in List/Fetch also.



-- 
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 #6740: NIFI-10868 Add PutDropbox processor

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


##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/PutDropbox.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxUploader;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.CommitInfo;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.UploadSessionAppendV2Uploader;
+import com.dropbox.core.v2.files.UploadSessionCursor;
+import com.dropbox.core.v2.files.UploadSessionFinishUploader;
+import com.dropbox.core.v2.files.UploadSessionStartUploader;
+import com.dropbox.core.v2.files.UploadUploader;
+import com.dropbox.core.v2.files.WriteMode;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+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.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+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.DataUnit;
+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;
+
+/**
+ * This processor uploads objects to Dropbox.
+ */
+@SeeAlso({ListDropbox.class, FetchDropbox.class})
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"dropbox", "storage", "put"})
+@CapabilityDescription("Puts content to a Dropbox folder.")
+@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the Dropbox object.")
+public class PutDropbox extends AbstractProcessor implements DropboxTrait {
+
+    public static final int UPLOAD_FILE_SIZE_LIMIT_IN_BYTES = 150 * 1024 * 1024;
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Files that have been successfully written to Dropbox are transferred to this relationship.")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Files that could not be written to Dropbox for some reason are transferred to this relationship.")
+            .build();
+
+    public static final PropertyDescriptor FOLDER = new PropertyDescriptor.Builder()
+            .name("folder")
+            .displayName("Folder")
+            .description("The Dropbox identifier or path of the folder to upload files to. "
+                    + "The folder will be created if it does not exist yet.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("/.*|id:.*")))
+            .defaultValue("/")
+            .build();
+
+    public static final PropertyDescriptor FILE_NAME = new PropertyDescriptor.Builder()
+            .name("file-name")
+            .displayName("File Name")
+            .description("The full name of the file to upload.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${filename}")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor UPLOAD_CHUNK_SIZE = new PropertyDescriptor.Builder()
+            .name("upload-chunk-size")
+            .displayName("Upload Chunk Size")
+            .description("The chunk size used to upload files larger than 150 MB in smaller parts (chunks). "
+                    + "It is recommended to specify chunk size as multiples of 4 MB.")
+            .defaultValue("8 MB")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createDataSizeBoundsValidator(1, UPLOAD_FILE_SIZE_LIMIT_IN_BYTES))
+            .required(false)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            CREDENTIAL_SERVICE,
+            FOLDER,
+            FILE_NAME,
+            UPLOAD_CHUNK_SIZE,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxySpec.HTTP_AUTH)
+    ));
+
+    private static final Set<Relationship> relationships;
+
+    static {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    private DbxClientV2 dropboxApiClient;
+    private DbxUploader dbxUploader;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        String dropboxClientId = format("%s-%s", getClass().getSimpleName(), getIdentifier());
+        dropboxApiClient = getDropboxApiClient(context, proxyConfiguration, dropboxClientId);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        String folder = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
+        String filename = context.getProperty(FILE_NAME).evaluateAttributeExpressions(flowFile).getValue();
+
+        FileMetadata uploadedFileMetadata = null;
+
+        long size = flowFile.getSize();
+        String uploadPath = convertFolderName(folder) + "/" + filename;
+
+        if (size > 0) {
+            try (final InputStream rawIn = session.read(flowFile)) {
+                if (size <= getUploadFileSizeLimit()) {
+                    dbxUploader = dropboxApiClient.files()
+                            .upload(uploadPath);
+                    uploadedFileMetadata = ((UploadUploader) dbxUploader).uploadAndFinish(rawIn);
+                } else {
+                    long chunkSize = context.getProperty(UPLOAD_CHUNK_SIZE)
+                            .evaluateAttributeExpressions()
+                            .asDataSize(DataUnit.B)
+                            .longValue();
+
+                    uploadedFileMetadata = uploadLargeFileInChunks(rawIn, size, chunkSize, uploadPath);
+                }
+            } catch (Exception e) {
+                getLogger().error("Exception occurred while uploading file '{}' to Dropbox folder '{}'", filename, folder, e);
+            }
+
+            if (uploadedFileMetadata != null) {
+                session.transfer(flowFile, REL_SUCCESS);
+            } else {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+        }
+    }
+
+    @OnUnscheduled
+    @OnDisabled
+    public synchronized void shutdown() {
+        if (dbxUploader != null) {
+            dbxUploader.close();
+        }
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    protected long getUploadFileSizeLimit() {
+        return UPLOAD_FILE_SIZE_LIMIT_IN_BYTES;
+    }

Review Comment:
   It was used for testing, removed it as the chunk limit can be set by the new property (CHUNK_UPLOAD_LIMIT).



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