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/05 16:16:08 UTC

[GitHub] [nifi] turcsanyip commented on a diff in pull request #6504: NIFI-10618: Add Asana connector

turcsanyip commented on code in PR #6504:
URL: https://github.com/apache/nifi/pull/6504#discussion_r1039539797


##########
nifi-nar-bundles/nifi-asana-bundle/nifi-asana-processors/src/main/java/org/apache/nifi/processors/asana/utils/PollableAsanaObjectFetcher.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.asana.utils;
+
+import static java.util.Collections.emptyIterator;
+
+import java.util.Iterator;
+
+public abstract class PollableAsanaObjectFetcher implements AsanaObjectFetcher {
+
+    private Iterator<AsanaObject> pending;
+
+    public PollableAsanaObjectFetcher() {
+        pending = emptyIterator();
+    }
+
+    @Override
+    public AsanaObject fetchNext() {
+        if (!pending.hasNext()) {
+            pending = poll();
+        }
+        return pending.hasNext() ? pending.next() : null;
+    }
+
+    protected abstract Iterator<AsanaObject> poll();

Review Comment:
   The current call chain is `fetchNext()` => `poll()` => `refreshObjects()` => `fetchXXX()`.
   I would somehow try to consolidate these various names because they refer to the same process.
   E.g. `fetchNext()` => `fetch()` => `fetchObjects()` => `fetchXXX()`
   
   Not sure but `Pollable-` may not be relevant any more due to the on-the-fly processing and also because it should process a single iteration. Simply `Abstract-` might be a better name.



##########
nifi-nar-bundles/nifi-asana-bundle/nifi-asana-processors/src/main/java/org/apache/nifi/processors/asana/utils/PollableAsanaObjectFetcher.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.asana.utils;
+
+import static java.util.Collections.emptyIterator;
+
+import java.util.Iterator;
+
+public abstract class PollableAsanaObjectFetcher implements AsanaObjectFetcher {
+
+    private Iterator<AsanaObject> pending;
+
+    public PollableAsanaObjectFetcher() {
+        pending = emptyIterator();
+    }
+
+    @Override
+    public AsanaObject fetchNext() {
+        if (!pending.hasNext()) {
+            pending = poll();
+        }
+        return pending.hasNext() ? pending.next() : null;
+    }

Review Comment:
   The `Iterator` should be retrieved only once per `onTrigger` call. The returned `Iterator` can iterate over all the currently available Asana objects.
   
   Now we have a second call (when the first `Iterator` has no more elements) but it unnecessarily queries the same Asana objects again which items will be filtered out on the NiFi side.



##########
nifi-nar-bundles/nifi-asana-bundle/nifi-asana-processors/src/main/java/org/apache/nifi/processors/asana/GetAsanaObject.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.asana;
+
+import static java.lang.String.format;
+import static java.lang.String.join;
+import static java.util.Collections.singletonMap;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_EVENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_MEMBERS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_STATUS_ATTACHMENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_STATUS_UPDATES;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_STORIES;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TASKS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TASK_ATTACHMENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TEAM_MEMBERS;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.http.entity.ContentType;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.controller.asana.AsanaClient;
+import org.apache.nifi.controller.asana.AsanaClientProviderService;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.asana.utils.AsanaObject;
+import org.apache.nifi.processors.asana.utils.AsanaObjectFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaObjectState;
+import org.apache.nifi.processors.asana.utils.AsanaProjectEventFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectMembershipFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectStatusAttachmentFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectStatusFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaStoryFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTagFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTaskAttachmentFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTaskFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTeamFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTeamMemberFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaUserFetcher;
+import org.apache.nifi.reporting.InitializationException;
+
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@WritesAttribute(attribute = GetAsanaObject.ASANA_GID, description = "Global ID of the object in Asana.")
+@Tags({"asana", "source", "ingest"})
+@CapabilityDescription("This processor collects data from Asana")
+public class GetAsanaObject extends AbstractProcessor {

Review Comment:
   Though the processor stores its state in an external cache but it is loaded int memory while the onTrigger is running. This state object can consume high memory and for this reason please add `@SystemResourceConsideration` annotation with `SystemResource.MEMORY`.



##########
nifi-nar-bundles/nifi-asana-bundle/nifi-asana-processors/src/main/java/org/apache/nifi/processors/asana/GetAsanaObject.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.asana;
+
+import static java.lang.String.format;
+import static java.lang.String.join;
+import static java.util.Collections.singletonMap;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_EVENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_MEMBERS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_STATUS_ATTACHMENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_STATUS_UPDATES;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_STORIES;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TASKS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TASK_ATTACHMENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TEAM_MEMBERS;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.http.entity.ContentType;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.controller.asana.AsanaClient;
+import org.apache.nifi.controller.asana.AsanaClientProviderService;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.asana.utils.AsanaObject;
+import org.apache.nifi.processors.asana.utils.AsanaObjectFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaObjectState;
+import org.apache.nifi.processors.asana.utils.AsanaProjectEventFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectMembershipFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectStatusAttachmentFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectStatusFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaStoryFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTagFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTaskAttachmentFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTaskFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTeamFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTeamMemberFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaUserFetcher;
+import org.apache.nifi.reporting.InitializationException;
+
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@WritesAttribute(attribute = GetAsanaObject.ASANA_GID, description = "Global ID of the object in Asana.")
+@Tags({"asana", "source", "ingest"})
+@CapabilityDescription("This processor collects data from Asana")
+public class GetAsanaObject extends AbstractProcessor {
+
+    protected static final String ASANA_GID = "asana.gid";
+    protected static final String ASANA_CLIENT_SERVICE = "asana-controller-service";
+    protected static final String DISTRIBUTED_CACHE_SERVICE = "distributed-cache-service";
+    protected static final String ASANA_OBJECT_TYPE = "asana-object-type";
+    protected static final String ASANA_PROJECT_NAME = "asana-project-name";
+    protected static final String ASANA_SECTION_NAME = "asana-section-name";
+    protected static final String ASANA_TAG_NAME = "asana-tag-name";
+    protected static final String ASANA_TEAM_NAME = "asana-team-name";
+    protected static final String ASANA_OUTPUT_BATCH_SIZE = "asana-output-batch-size";
+    protected static final String REL_NAME_NEW = "new";
+    protected static final String REL_NAME_UPDATED = "updated";
+    protected static final String REL_NAME_REMOVED = "removed";
+
+    protected static final PropertyDescriptor PROP_ASANA_CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name(ASANA_CLIENT_SERVICE)
+            .displayName("Asana Client Service")
+            .description("Specify which controller service to use for accessing Asana.")
+            .required(true)
+            .identifiesControllerService(AsanaClientProviderService.class)
+            .build();
+
+    protected static final PropertyDescriptor PROP_DISTRIBUTED_CACHE_SERVICE = new Builder()
+            .name(DISTRIBUTED_CACHE_SERVICE)
+            .displayName("Distributed Cache Service")
+            .description("Cache service to store fetched item fingerprints. These, from the last successful query"
+                    + " are stored, in order to enable incremental loading and change detection.")
+            .required(true)
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_OBJECT_TYPE = new PropertyDescriptor.Builder()
+            .name(ASANA_OBJECT_TYPE)
+            .displayName("Object Type")
+            .description("Specify what kind of objects to be collected from Asana")
+            .required(true)
+            .allowableValues(AsanaObjectType.class)
+            .defaultValue(AV_COLLECT_TASKS.getValue())
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_PROJECT = new PropertyDescriptor.Builder()
+            .name(ASANA_PROJECT_NAME)
+            .displayName("Project Name")
+            .description("Fetch only objects in this project. Case sensitive.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(
+                PROP_ASANA_OBJECT_TYPE,
+                    AV_COLLECT_TASKS,
+                    AV_COLLECT_TASK_ATTACHMENTS,
+                    AV_COLLECT_PROJECT_MEMBERS,
+                    AV_COLLECT_STORIES,
+                    AV_COLLECT_PROJECT_STATUS_UPDATES,
+                    AV_COLLECT_PROJECT_STATUS_ATTACHMENTS,
+                    AV_COLLECT_PROJECT_EVENTS)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_SECTION = new PropertyDescriptor.Builder()
+            .name(ASANA_SECTION_NAME)
+            .displayName("Section Name")
+            .description("Fetch only objects in this section. Case sensitive.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(PROP_ASANA_OBJECT_TYPE,
+                    AV_COLLECT_TASKS,
+                    AV_COLLECT_TASK_ATTACHMENTS,
+                    AV_COLLECT_STORIES)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_TAG = new PropertyDescriptor.Builder()
+            .name(ASANA_TAG_NAME)
+            .displayName("Tag")
+            .description("Fetch only objects having this tag. Case sensitive.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(PROP_ASANA_OBJECT_TYPE,
+                    AV_COLLECT_TASKS,
+                    AV_COLLECT_TASK_ATTACHMENTS,
+                    AV_COLLECT_STORIES)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_TEAM_NAME = new PropertyDescriptor.Builder()
+            .name(ASANA_TEAM_NAME)
+            .displayName("Team")
+            .description("Team name. Case sensitive.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(PROP_ASANA_OBJECT_TYPE, AV_COLLECT_TEAM_MEMBERS)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name(ASANA_OUTPUT_BATCH_SIZE)
+            .displayName("Output Batch Size")
+            .description("The number of items batched together in a single Flow File. If set to 1 (default), then each item is"
+                    + " transferred in a separate Flow File and each will have an asana.gid attribute, to help identifying"
+                    + " the fetched item on the server side, if needed. If the batch size is greater than 1, then the"
+                    + " specified amount of items are batched together in a single Flow File as a Json array, and the"
+                    + " Flow Files won't have the asana.gid attribute.")
+            .defaultValue("1")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    protected static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            PROP_ASANA_CLIENT_SERVICE,
+            PROP_DISTRIBUTED_CACHE_SERVICE,
+            PROP_ASANA_OBJECT_TYPE,
+            PROP_ASANA_PROJECT,
+            PROP_ASANA_SECTION,
+            PROP_ASANA_TEAM_NAME,
+            PROP_ASANA_TAG,
+            PROP_ASANA_OUTPUT_BATCH_SIZE
+    ));
+
+    protected static final Relationship REL_NEW = new Relationship.Builder()
+            .name(REL_NAME_NEW)
+            .description("Newly collected objects are routed to this relationship.")
+            .build();
+
+    protected static final Relationship REL_UPDATED = new Relationship.Builder()
+            .name(REL_NAME_UPDATED)
+            .description("Objects that have already been collected earlier, but were updated since, are routed to this relationship.")
+            .build();
+
+    protected static final Relationship REL_REMOVED = new Relationship.Builder()
+            .name(REL_NAME_REMOVED)
+            .description("Notification about deleted objects are routed to this relationship. "
+                    + "Flow files will not have any payload. IDs of the resources no longer exist "
+                    + "are carried by the asana.gid attribute of the generated FlowFiles.")
+            .build();
+
+    protected static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_NEW,
+            REL_UPDATED,
+            REL_REMOVED
+    )));
+    protected static final GenericObjectSerDe<String> STATE_MAP_KEY_SERIALIZER = new GenericObjectSerDe<>();
+    protected static final GenericObjectSerDe<Map<String, String>> STATE_MAP_VALUE_SERIALIZER = new GenericObjectSerDe<>();
+
+    private volatile AsanaObjectFetcher objectFetcher;
+    private volatile Integer batchSize;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws InitializationException {
+        AsanaClientProviderService controllerService = context.getProperty(PROP_ASANA_CLIENT_SERVICE).asControllerService(AsanaClientProviderService.class);
+        AsanaClient client = controllerService.createClient();
+        batchSize = context.getProperty(PROP_ASANA_OUTPUT_BATCH_SIZE).asInteger();
+
+        try {
+            getLogger().debug("Initializing object fetcher...");
+            objectFetcher = createObjectFetcher(context, client);
+        } catch (Exception e) {
+            throw new InitializationException(e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        try {
+            Map<String, String> state = recoverState(context).orElse(Collections.emptyMap());
+            getLogger().debug("Attempting to load state: {}", state);
+            objectFetcher.loadState(state);
+        } catch (Exception e) {
+            getLogger().info("Failed to recover state. Falling back to clean start.");
+            objectFetcher.clearState();
+        }

Review Comment:
   I think we should distinguish errors coming from `recoverState()` versus `loadState()`.
   `recoverState()` can throw an exception if the external cache is temporarily not available but in this case we should not clear the state but rather log the error and do nothing until the cache issue is fixed.
   `loadState()` errors seem to be unrecoverable errors and clean start is reasonable for this case.



##########
nifi-nar-bundles/nifi-asana-bundle/nifi-asana-processors/src/main/resources/docs/org.apache.nifi.processors.asana.GetAsanaObject/additionalDetails.html:
##########
@@ -0,0 +1,100 @@
+<!DOCTYPE html>
+<html lang="en" xmlns="http://www.w3.org/1999/html">
+<!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+
+<head>
+    <meta charset="utf-8"/>
+    <title>GetAsanaObject</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+    <style>
+        h2 {margin-top: 4em}
+        h3 {margin-top: 3em}
+        td {text-align: left}
+    </style>
+</head>
+
+<body>
+
+<h1>GetAsanaObject</h1>
+
+<h3>Description</h3>
+<p>
+    This processor collects various objects (eg. tasks, comments, etc...) from Asana via the specified
+    <code>AsanaClientService</code>. When the processor started for the first time with a given configuration
+    it collects each of the objects matching the user specified criteria, and emits <code>FlowFile</code>s
+    of each on the <code>NEW</code> relationship. Then, it polls Asana in the frequency of the configured <em>Yield</em>
+    duration, and detects changes by comparing the object fingerprints. When there are updates, it emits them through

Review Comment:
   ```suggestion
       of each on the <code>NEW</code> relationship. Then, it polls Asana in the frequency of the configured <em>Run Schedule</em>
       and detects changes by comparing the object fingerprints. When there are updates, it emits them through
   ```
   The base frequency is `Run Schedule`. `Yield Duration` takes effect only if there were no changed items in the previous run (and `Run Schedule < Yield Duration`).



##########
nifi-nar-bundles/nifi-asana-bundle/nifi-asana-processors/src/main/java/org/apache/nifi/processors/asana/GetAsanaObject.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * 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.asana;
+
+import static java.lang.String.format;
+import static java.lang.String.join;
+import static java.util.Collections.singletonMap;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_EVENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_MEMBERS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_STATUS_ATTACHMENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_PROJECT_STATUS_UPDATES;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_STORIES;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TASKS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TASK_ATTACHMENTS;
+import static org.apache.nifi.processors.asana.AsanaObjectType.AV_COLLECT_TEAM_MEMBERS;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.http.entity.ContentType;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.controller.asana.AsanaClient;
+import org.apache.nifi.controller.asana.AsanaClientProviderService;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.asana.utils.AsanaObject;
+import org.apache.nifi.processors.asana.utils.AsanaObjectFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaObjectState;
+import org.apache.nifi.processors.asana.utils.AsanaProjectEventFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectMembershipFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectStatusAttachmentFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaProjectStatusFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaStoryFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTagFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTaskAttachmentFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTaskFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTeamFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaTeamMemberFetcher;
+import org.apache.nifi.processors.asana.utils.AsanaUserFetcher;
+import org.apache.nifi.reporting.InitializationException;
+
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@WritesAttribute(attribute = GetAsanaObject.ASANA_GID, description = "Global ID of the object in Asana.")
+@Tags({"asana", "source", "ingest"})
+@CapabilityDescription("This processor collects data from Asana")
+public class GetAsanaObject extends AbstractProcessor {
+
+    protected static final String ASANA_GID = "asana.gid";
+    protected static final String ASANA_CLIENT_SERVICE = "asana-controller-service";
+    protected static final String DISTRIBUTED_CACHE_SERVICE = "distributed-cache-service";
+    protected static final String ASANA_OBJECT_TYPE = "asana-object-type";
+    protected static final String ASANA_PROJECT_NAME = "asana-project-name";
+    protected static final String ASANA_SECTION_NAME = "asana-section-name";
+    protected static final String ASANA_TAG_NAME = "asana-tag-name";
+    protected static final String ASANA_TEAM_NAME = "asana-team-name";
+    protected static final String ASANA_OUTPUT_BATCH_SIZE = "asana-output-batch-size";
+    protected static final String REL_NAME_NEW = "new";
+    protected static final String REL_NAME_UPDATED = "updated";
+    protected static final String REL_NAME_REMOVED = "removed";
+
+    protected static final PropertyDescriptor PROP_ASANA_CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name(ASANA_CLIENT_SERVICE)
+            .displayName("Asana Client Service")
+            .description("Specify which controller service to use for accessing Asana.")
+            .required(true)
+            .identifiesControllerService(AsanaClientProviderService.class)
+            .build();
+
+    protected static final PropertyDescriptor PROP_DISTRIBUTED_CACHE_SERVICE = new Builder()
+            .name(DISTRIBUTED_CACHE_SERVICE)
+            .displayName("Distributed Cache Service")
+            .description("Cache service to store fetched item fingerprints. These, from the last successful query"
+                    + " are stored, in order to enable incremental loading and change detection.")
+            .required(true)
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_OBJECT_TYPE = new PropertyDescriptor.Builder()
+            .name(ASANA_OBJECT_TYPE)
+            .displayName("Object Type")
+            .description("Specify what kind of objects to be collected from Asana")
+            .required(true)
+            .allowableValues(AsanaObjectType.class)
+            .defaultValue(AV_COLLECT_TASKS.getValue())
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_PROJECT = new PropertyDescriptor.Builder()
+            .name(ASANA_PROJECT_NAME)
+            .displayName("Project Name")
+            .description("Fetch only objects in this project. Case sensitive.")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(
+                PROP_ASANA_OBJECT_TYPE,
+                    AV_COLLECT_TASKS,
+                    AV_COLLECT_TASK_ATTACHMENTS,
+                    AV_COLLECT_PROJECT_MEMBERS,
+                    AV_COLLECT_STORIES,
+                    AV_COLLECT_PROJECT_STATUS_UPDATES,
+                    AV_COLLECT_PROJECT_STATUS_ATTACHMENTS,
+                    AV_COLLECT_PROJECT_EVENTS)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_SECTION = new PropertyDescriptor.Builder()
+            .name(ASANA_SECTION_NAME)
+            .displayName("Section Name")
+            .description("Fetch only objects in this section. Case sensitive.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(PROP_ASANA_OBJECT_TYPE,
+                    AV_COLLECT_TASKS,
+                    AV_COLLECT_TASK_ATTACHMENTS,
+                    AV_COLLECT_STORIES)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_TAG = new PropertyDescriptor.Builder()
+            .name(ASANA_TAG_NAME)
+            .displayName("Tag")
+            .description("Fetch only objects having this tag. Case sensitive.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(PROP_ASANA_OBJECT_TYPE,
+                    AV_COLLECT_TASKS,
+                    AV_COLLECT_TASK_ATTACHMENTS,
+                    AV_COLLECT_STORIES)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_TEAM_NAME = new PropertyDescriptor.Builder()
+            .name(ASANA_TEAM_NAME)
+            .displayName("Team")
+            .description("Team name. Case sensitive.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(PROP_ASANA_OBJECT_TYPE, AV_COLLECT_TEAM_MEMBERS)
+            .build();
+
+    protected static final PropertyDescriptor PROP_ASANA_OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name(ASANA_OUTPUT_BATCH_SIZE)
+            .displayName("Output Batch Size")
+            .description("The number of items batched together in a single Flow File. If set to 1 (default), then each item is"
+                    + " transferred in a separate Flow File and each will have an asana.gid attribute, to help identifying"
+                    + " the fetched item on the server side, if needed. If the batch size is greater than 1, then the"
+                    + " specified amount of items are batched together in a single Flow File as a Json array, and the"
+                    + " Flow Files won't have the asana.gid attribute.")
+            .defaultValue("1")
+            .required(true)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .build();
+
+    protected static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            PROP_ASANA_CLIENT_SERVICE,
+            PROP_DISTRIBUTED_CACHE_SERVICE,
+            PROP_ASANA_OBJECT_TYPE,
+            PROP_ASANA_PROJECT,
+            PROP_ASANA_SECTION,
+            PROP_ASANA_TEAM_NAME,
+            PROP_ASANA_TAG,
+            PROP_ASANA_OUTPUT_BATCH_SIZE
+    ));
+
+    protected static final Relationship REL_NEW = new Relationship.Builder()
+            .name(REL_NAME_NEW)
+            .description("Newly collected objects are routed to this relationship.")
+            .build();
+
+    protected static final Relationship REL_UPDATED = new Relationship.Builder()
+            .name(REL_NAME_UPDATED)
+            .description("Objects that have already been collected earlier, but were updated since, are routed to this relationship.")
+            .build();
+
+    protected static final Relationship REL_REMOVED = new Relationship.Builder()
+            .name(REL_NAME_REMOVED)
+            .description("Notification about deleted objects are routed to this relationship. "
+                    + "Flow files will not have any payload. IDs of the resources no longer exist "
+                    + "are carried by the asana.gid attribute of the generated FlowFiles.")
+            .build();
+
+    protected static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_NEW,
+            REL_UPDATED,
+            REL_REMOVED
+    )));
+    protected static final GenericObjectSerDe<String> STATE_MAP_KEY_SERIALIZER = new GenericObjectSerDe<>();
+    protected static final GenericObjectSerDe<Map<String, String>> STATE_MAP_VALUE_SERIALIZER = new GenericObjectSerDe<>();
+
+    private volatile AsanaObjectFetcher objectFetcher;
+    private volatile Integer batchSize;
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws InitializationException {
+        AsanaClientProviderService controllerService = context.getProperty(PROP_ASANA_CLIENT_SERVICE).asControllerService(AsanaClientProviderService.class);
+        AsanaClient client = controllerService.createClient();
+        batchSize = context.getProperty(PROP_ASANA_OUTPUT_BATCH_SIZE).asInteger();
+
+        try {
+            getLogger().debug("Initializing object fetcher...");
+            objectFetcher = createObjectFetcher(context, client);
+        } catch (Exception e) {
+            throw new InitializationException(e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        try {
+            Map<String, String> state = recoverState(context).orElse(Collections.emptyMap());
+            getLogger().debug("Attempting to load state: {}", state);
+            objectFetcher.loadState(state);

Review Comment:
   `AsanaObjectFetcher`'s state is kept in memory until the next `onTrigger()` call. It should be cleared at the end of `onTrigger()` in order to free up the resources in the fetcher object (like the `lastFingerprints` map). Or the fetcher object can be discarded and recreated in every `onTrigger`. 



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