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 19:52:51 UTC

[GitHub] [nifi] turcsanyip commented on a diff in pull request #6740: NIFI-10868 Add PutDropbox processor

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