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/08/16 15:40:23 UTC

[GitHub] [nifi] Lehel44 opened a new pull request, #6303: NIFI-10213: Create GetShopify processor

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

   <!-- Licensed to the Apache Software Foundation (ASF) under one or more -->
   <!-- contributor license agreements.  See the NOTICE file distributed with -->
   <!-- this work for additional information regarding copyright ownership. -->
   <!-- The ASF licenses this file to You under the Apache License, Version 2.0 -->
   <!-- (the "License"); you may not use this file except in compliance with -->
   <!-- the License.  You may obtain a copy of the License at -->
   <!--     http://www.apache.org/licenses/LICENSE-2.0 -->
   <!-- Unless required by applicable law or agreed to in writing, software -->
   <!-- distributed under the License is distributed on an "AS IS" BASIS, -->
   <!-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -->
   <!-- See the License for the specific language governing permissions and -->
   <!-- limitations under the License. -->
   
   # Summary
   
   [NIFI-10213](https://issues.apache.org/jira/browse/NIFI-10213)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI-10213) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


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

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

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


[GitHub] [nifi] Lehel44 commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters" +
+                " can be found in additional details.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    static final PropertyDescriptor STORE_DOMAIN = new PropertyDescriptor.Builder()
+            .name("store-domain")
+            .displayName("Store Domain")
+            .description("The domain of the Shopify store, e.g. nifistore.myshopify.com")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Access Token")
+            .description("Access Token to authenticate requests")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The Shopify REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor OBJECT_CATEGORY = new PropertyDescriptor.Builder()
+            .name("object-category")
+            .displayName("Object Category")
+            .description("Shopify object category")
+            .required(true)
+            .allowableValues(ResourceType.class)
+            .build();
+
+    static final PropertyDescriptor RESULT_LIMIT = new PropertyDescriptor.Builder()
+            .name("result-limit")
+            .displayName("Result Limit")
+            .description("The maximum number of results to request for each invocation of the Processor")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.createLongValidator(1, 100, true))
+            .build();
+
+    static final PropertyDescriptor IS_INCREMENTAL = new PropertyDescriptor.Builder()
+            .name("is-incremental")
+            .displayName("Incremental Loading")
+            .description("The processor can incrementally load the queried objects so that each object is queried exactly once." +
+                    " For each query, the processor queries objects which were created or modified after the previous run time" +
+                    " but before the current time.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    static final PropertyDescriptor INCREMENTAL_DELAY = new PropertyDescriptor.Builder()
+            .name("incremental-delay")
+            .displayName("Incremental Delay")
+            .description("The ending timestamp of the time window will be adjusted earlier by the amount configured in this property." +
+                    " For example, with a property value of 10 seconds, an ending timestamp of 12:30:45 would be changed to 12:30:35.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    static final PropertyDescriptor INITIAL_INCREMENTAL_FILTER = new PropertyDescriptor.Builder()
+            .name("initial-incremental-filter")
+            .displayName("Initial Incremental Start Time")
+            .description("This property specifies the start time as Epoch Time that the processor applies when running the first request.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Controller service for HTTP client operations")
+            .required(true)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getPropertyDisplayName())
+                            .description(resourceType.getPropertyDescription())
+                            .required(true)
+                            .dependsOn(OBJECT_CATEGORY, resourceType.getValue())
+                            .allowableValues(resourceType.getResourcesAsAllowableValues())
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                STORE_DOMAIN,
+                ACCESS_TOKEN,
+                API_VERSION,
+                OBJECT_CATEGORY
+        ));
+        propertyDescriptors.addAll(resourceDescriptors);
+        propertyDescriptors.addAll(Arrays.asList(
+                RESULT_LIMIT,
+                IS_INCREMENTAL,
+                INCREMENTAL_DELAY,
+                INITIAL_INCREMENTAL_FILTER,
+                WEB_CLIENT_PROVIDER)
+        );
+        return Collections.unmodifiableList(propertyDescriptors);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Pattern CURSOR_PATTERN = Pattern.compile("<([^<]*)>; rel=\"next\"");
+
+    private volatile ShopifyRestService shopifyRestService;
+    private volatile ShopifyResource shopifyResource;
+    private volatile String resourceName;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final WebClientServiceProvider webClientServiceProvider =
+                context.getProperty(WEB_CLIENT_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        final WebClientService webClientService = webClientServiceProvider.getWebClientService();
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder();
+
+        final String apiVersion = context.getProperty(API_VERSION).getValue();
+        final String baseUrl = context.getProperty(STORE_DOMAIN).getValue();
+        final String accessToken = context.getProperty(ACCESS_TOKEN).getValue();
+
+        final String category = context.getProperty(OBJECT_CATEGORY).getValue();
+        final ResourceType resourceType = ResourceType.valueOf(category);
+        resourceName = context.getProperty(propertyMap.get(resourceType)).getValue();
+        final String limit = context.getProperty(RESULT_LIMIT).getValue();
+
+        shopifyResource = resourceType.getResource(resourceName);
+
+        shopifyRestService = getShopifyRestService(webClientService, uriBuilder, apiVersion, baseUrl, accessToken, resourceName,
+                limit, shopifyResource.getIncrementalLoadingParameter());
+    }
+
+    ShopifyRestService getShopifyRestService(final WebClientService webClientService, final HttpUriBuilder uriBuilder,
+                                             final String apiVersion, final String baseUrl, final String accessToken, final String resourceName,
+                                             final String limit, final IncrementalLoadingParameter incrementalLoadingParameter) {
+        return new ShopifyRestService(
+                webClientService,
+                uriBuilder,
+                apiVersion,
+                baseUrl,
+                accessToken,
+                resourceName,
+                limit,
+                incrementalLoadingParameter
+        );
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final StateMap state = getState(context);
+        final boolean isIncremental = context.getProperty(IS_INCREMENTAL).asBoolean();
+        final String initialStartTime = context.getProperty(INITIAL_INCREMENTAL_FILTER).getValue();
+        final Long incrDelayMs = context.getProperty(INCREMENTAL_DELAY).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        String lastExecutionTime = state.get(resourceName);
+        if (lastExecutionTime == null && initialStartTime != null) {
+            lastExecutionTime = initialStartTime;
+        }
+
+        Instant now = getCurrentExecutionTime();
+        if (incrDelayMs != null) {
+            now = now.minus(incrDelayMs, ChronoUnit.MILLIS);
+        }
+        final String currentExecutionTime = now.toString();
+
+        String cursor = null;
+        do {
+            final AtomicInteger objectCountHolder = new AtomicInteger();
+            try (HttpResponseEntity response = shopifyRestService.getShopifyObjects(isIncremental, lastExecutionTime, currentExecutionTime, cursor)) {
+                cursor = getPageCursor(response);
+                if (response.statusCode() == HttpResponseStatus.OK.getCode()) {
+                    FlowFile flowFile = session.create();
+                    flowFile = session.write(flowFile, parseHttpResponse(response, objectCountHolder));
+                    if (objectCountHolder.get() > 0) {
+                        session.transfer(flowFile, REL_SUCCESS);
+                        if (cursor == null && isIncremental) {

Review Comment:
   I think this check is at the wrong place. This branch means no pagination but to store the processing time in the state. I think we don't need the isPaginationSupported anymore since the while (cursor != null); will check if a cursor URL is returned from the response header.



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

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

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/pom.xml:
##########
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-shopify-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-shopify-processors</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>

Review Comment:
   This dependency can be also moved under the  "Test dependencies" line



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

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

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


[GitHub] [nifi] Lehel44 commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters" +
+                " can be found in additional details.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    static final PropertyDescriptor STORE_DOMAIN = new PropertyDescriptor.Builder()
+            .name("store-domain")
+            .displayName("Store Domain")
+            .description("The domain of the Shopify store, e.g. nifistore.myshopify.com")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Access Token")
+            .description("Access Token to authenticate requests")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The Shopify REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor OBJECT_CATEGORY = new PropertyDescriptor.Builder()
+            .name("object-category")
+            .displayName("Object Category")
+            .description("Shopify object category")
+            .required(true)
+            .allowableValues(ResourceType.class)
+            .build();
+
+    static final PropertyDescriptor RESULT_LIMIT = new PropertyDescriptor.Builder()
+            .name("result-limit")
+            .displayName("Result Limit")
+            .description("The maximum number of results to request for each invocation of the Processor")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.createLongValidator(1, 100, true))
+            .build();
+
+    static final PropertyDescriptor IS_INCREMENTAL = new PropertyDescriptor.Builder()
+            .name("is-incremental")
+            .displayName("Incremental Loading")
+            .description("The processor can incrementally load the queried objects so that each object is queried exactly once." +
+                    " For each query, the processor queries objects which were created or modified after the previous run time" +
+                    " but before the current time.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    static final PropertyDescriptor INCREMENTAL_DELAY = new PropertyDescriptor.Builder()
+            .name("incremental-delay")
+            .displayName("Incremental Delay")
+            .description("The ending timestamp of the time window will be adjusted earlier by the amount configured in this property." +
+                    " For example, with a property value of 10 seconds, an ending timestamp of 12:30:45 would be changed to 12:30:35.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    static final PropertyDescriptor INITIAL_INCREMENTAL_FILTER = new PropertyDescriptor.Builder()
+            .name("initial-incremental-filter")
+            .displayName("Initial Incremental Start Time")
+            .description("This property specifies the start time as Epoch Time that the processor applies when running the first request.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Controller service for HTTP client operations")
+            .required(true)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getPropertyDisplayName())
+                            .description(resourceType.getPropertyDescription())
+                            .required(true)
+                            .dependsOn(OBJECT_CATEGORY, resourceType.getValue())
+                            .allowableValues(resourceType.getResourcesAsAllowableValues())
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                STORE_DOMAIN,
+                ACCESS_TOKEN,
+                API_VERSION,
+                OBJECT_CATEGORY
+        ));
+        propertyDescriptors.addAll(resourceDescriptors);
+        propertyDescriptors.addAll(Arrays.asList(
+                RESULT_LIMIT,
+                IS_INCREMENTAL,
+                INCREMENTAL_DELAY,
+                INITIAL_INCREMENTAL_FILTER,
+                WEB_CLIENT_PROVIDER)
+        );
+        return Collections.unmodifiableList(propertyDescriptors);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Pattern CURSOR_PATTERN = Pattern.compile("<([^<]*)>; rel=\"next\"");
+
+    private volatile ShopifyRestService shopifyRestService;
+    private volatile ShopifyResource shopifyResource;
+    private volatile String resourceName;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final WebClientServiceProvider webClientServiceProvider =
+                context.getProperty(WEB_CLIENT_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        final WebClientService webClientService = webClientServiceProvider.getWebClientService();
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder();
+
+        final String apiVersion = context.getProperty(API_VERSION).getValue();
+        final String baseUrl = context.getProperty(STORE_DOMAIN).getValue();
+        final String accessToken = context.getProperty(ACCESS_TOKEN).getValue();
+
+        final String category = context.getProperty(OBJECT_CATEGORY).getValue();
+        final ResourceType resourceType = ResourceType.valueOf(category);
+        resourceName = context.getProperty(propertyMap.get(resourceType)).getValue();
+        final String limit = context.getProperty(RESULT_LIMIT).getValue();
+
+        shopifyResource = resourceType.getResource(resourceName);
+
+        shopifyRestService = getShopifyRestService(webClientService, uriBuilder, apiVersion, baseUrl, accessToken, resourceName,
+                limit, shopifyResource.getIncrementalLoadingParameter());
+    }
+
+    ShopifyRestService getShopifyRestService(final WebClientService webClientService, final HttpUriBuilder uriBuilder,
+                                             final String apiVersion, final String baseUrl, final String accessToken, final String resourceName,
+                                             final String limit, final IncrementalLoadingParameter incrementalLoadingParameter) {
+        return new ShopifyRestService(
+                webClientService,
+                uriBuilder,
+                apiVersion,
+                baseUrl,
+                accessToken,
+                resourceName,
+                limit,
+                incrementalLoadingParameter
+        );
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final StateMap state = getState(context);
+        final boolean isIncremental = context.getProperty(IS_INCREMENTAL).asBoolean();
+        final String initialStartTime = context.getProperty(INITIAL_INCREMENTAL_FILTER).getValue();
+        final Long incrDelayMs = context.getProperty(INCREMENTAL_DELAY).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        String lastExecutionTime = state.get(resourceName);
+        if (lastExecutionTime == null && initialStartTime != null) {
+            lastExecutionTime = initialStartTime;
+        }
+
+        Instant now = getCurrentExecutionTime();
+        if (incrDelayMs != null) {
+            now = now.minus(incrDelayMs, ChronoUnit.MILLIS);
+        }
+        final String currentExecutionTime = now.toString();
+
+        String cursor = null;
+        do {
+            final AtomicInteger objectCountHolder = new AtomicInteger();
+            try (HttpResponseEntity response = shopifyRestService.getShopifyObjects(isIncremental, lastExecutionTime, currentExecutionTime, cursor)) {
+                cursor = getPageCursor(response);
+                if (response.statusCode() == HttpResponseStatus.OK.getCode()) {
+                    FlowFile flowFile = session.create();
+                    flowFile = session.write(flowFile, parseHttpResponse(response, objectCountHolder));
+                    if (objectCountHolder.get() > 0) {
+                        session.transfer(flowFile, REL_SUCCESS);
+                        if (cursor == null && isIncremental) {
+                            final Map<String, String> stateMap = new HashMap<>(state.toMap());
+                            stateMap.put(resourceName, currentExecutionTime);
+                            updateState(context, stateMap);
+                        }
+                    } else {
+                        getLogger().debug("Empty response when requested Shopify resource: [{}]", resourceName);
+                        session.remove(flowFile);
+                    }
+                } else if (response.statusCode() >= 400) {
+                    if (response.statusCode() == TOO_MANY_REQUESTS) {
+                        context.yield();
+                        throw new ProcessException(String.format(
+                                "Rate limit exceeded, yielding before retrying request. HTTP %d error for requested URI [%s]",
+                                response.statusCode(), resourceName));
+                    } else {
+                        context.yield();
+                        getLogger().warn("HTTP {} error for requested Shopify resource [{}]", response.statusCode(),
+                                resourceName);
+                    }
+                }

Review Comment:
   The 2xx response codes are successful ones. The 1xx informational and the 3xx are redirects could be handled as error in this case. What do you think of changing the error condition to 200 <= resp code < 300?



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

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

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.nifi.components.AllowableValue;
+
+public class ResourceDirectory {
+
+    private ResourceDirectory() {
+    }
+
+    private static final Map<ResourceType, List<ShopifyResource>> resourceMap;
+
+    static {
+        resourceMap = new EnumMap<>(ResourceType.class);
+        resourceMap.put(ResourceType.CUSTOMERS, getCustomerResources());
+        resourceMap.put(ResourceType.DISCOUNTS, getDiscountResources());
+        resourceMap.put(ResourceType.INVENTORY, getInventoryResources());
+        resourceMap.put(ResourceType.ONLINE_STORE, getOnlineStoreResources());
+        resourceMap.put(ResourceType.ORDERS, getOrderResources());
+        resourceMap.put(ResourceType.PRODUCT, getProductResources());
+        resourceMap.put(ResourceType.SALES_CHANNELS, getSalesChannelResources());
+        resourceMap.put(ResourceType.STORE_PROPERTIES, getStorePropertyResources());
+    }
+
+    private static List<ShopifyResource> getCustomerResources() {
+        final ShopifyResource customer = ShopifyResource.newInstance(
+                "customers",
+                "Customers",
+                "The Customer resource stores information about a shop's customers, such as their contact details," +
+                        " their order history, and whether they've agreed to receive email marketing.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource customerSavedSearch = ShopifyResource.newInstance(
+                "customer_saved_searches",
+                "Customer Saved Searches",
+                "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(customer, customerSavedSearch));
+    }
+
+    private static List<ShopifyResource> getDiscountResources() {
+        final ShopifyResource priceRule = ShopifyResource.newInstance(
+                "price_rules",
+                "Price Rules",
+                "The PriceRule resource can be used to get discounts using conditions",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.singletonList(priceRule);
+    }
+
+    private static List<ShopifyResource> getInventoryResources() {
+        final ShopifyResource location = ShopifyResource.newInstance(
+                "locations",
+                "Locations",
+                "A location represents a geographical location where your stores, pop-up stores, headquarters, and warehouses exist.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.singletonList(location);
+    }
+
+    private static List<ShopifyResource> getOnlineStoreResources() {
+        final ShopifyResource blog = ShopifyResource.newInstance(
+                "blogs",
+                "Blogs",
+                "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource comment = ShopifyResource.newInstance(
+                "comments",
+                "Comments",
+                "A comment is a reader's response to an article in a blog.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource page = ShopifyResource.newInstance(
+                "pages",
+                "Pages",
+                "Shopify stores come with a tool for creating basic HTML web pages.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource redirect = ShopifyResource.newInstance(
+                "redirects",
+                "Redirects",
+                "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource scriptTag = ShopifyResource.newInstance(
+                "script_tags",
+                "Script Tags",
+                "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource theme = ShopifyResource.newInstance(
+                "themes",
+                "Themes",
+                "A theme controls the look and feel of a Shopify online store.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(blog, comment, page, redirect, scriptTag, theme));
+    }
+
+    private static List<ShopifyResource> getOrderResources() {
+        final ShopifyResource abandonedCheckouts = ShopifyResource.newInstance(
+                "checkouts",
+                "Abandoned Checkouts",
+                "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource draftOrders = ShopifyResource.newInstance(
+                "draft_orders",
+                "Draft Orders",
+                "Merchants can use draft orders to create orders on behalf of their customers.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource orders = ShopifyResource.newInstance(
+                "orders",
+                "Orders",
+                "An order is a customer's request to purchase one or more products from a shop.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(abandonedCheckouts, draftOrders, orders));
+    }
+
+    private static List<ShopifyResource> getProductResources() {
+        final ShopifyResource collect = ShopifyResource.newInstance(
+                "collects",
+                "Collects",
+                "Collects are meant for managing the relationship between products and custom collections.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource customCollection = ShopifyResource.newInstance(
+                "custom_collections",
+                "Custom Collections",
+                "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource product = ShopifyResource.newInstance(
+                "products",
+                "Products",
+                "Get products in a merchant's store ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource smartCollection = ShopifyResource.newInstance(
+                "smart_collections",
+                "Smart Collections",
+                "A smart collection is a grouping of products defined by rules that are set by the merchant.",

Review Comment:
   The name of resource is sometimes written with small caps (e.g. "smart collection"), sometimes with camel case (e.g. "CollectionListing") in the description . 



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/pom.xml:
##########
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-shopify-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-shopify-processors</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>

Review Comment:
   This dependency can be also moved under the <!-- Test dependencies --> line



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

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

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


[GitHub] [nifi] tpalfy commented on pull request #6303: NIFI-10213: Create GetShopify processor

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

   LGTM
   Thanks for your work @Lehel44 !
   Merged to main.


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

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

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


[GitHub] [nifi] tpalfy commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/resources/docs/org.apache.nifi.shopify.processors.GetShopify/additionalDetails.html:
##########
@@ -0,0 +1,148 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>GetShopify</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<h2>Setting Up a Custom App</h2>
+<p>
+    Follow the <a href="https://help.shopify.com/en/manual/apps/custom-apps">Shopify tutorial</a> to enable and create
+    private apps, set API Scopes and generate API tokens.
+</p>
+<h2>Incremental Loading</h2>
+<p>
+    Some resources can be processed incrementally by NiFi. This means that only resources created or modified after the
+    last run time of the processor are displayed. The processor state can be reset in the context menu. The following
+    list shows which date-time fields are incremented for which resources.
+<ul>
+    <li>Access
+        <ul>
+            <li>Access Scope: none</li>
+            <li>StoreFront Access Token: none</li>
+        </ul>
+    </li>
+    <li>Analytics
+        <ul>
+            <li>Reports: updated_at_min</li>
+        </ul>
+    </li>
+    <li>Billing
+        <ul>
+            <li>Application Charge: none</li>
+            <li>Application Credit: none</li>
+            <li>Recurring Application Charge: none</li>
+        </ul>
+    </li>
+    <li>Customers
+        <ul>
+            <li>Customers: updated_at_min</li>
+            <li>Customer Saved Searches: none</li>
+        </ul>
+    </li>
+    <li>Discounts
+        <ul>
+            <li>Price Rules: updated_at_min</li>
+        </ul>
+    </li>
+    <li>Events
+        <ul>
+            <li>Events: created_at_min</li>
+        </ul>
+    </li>
+    <li>Inventory
+        <ul>
+            <li>Inventory Levels: updated_at_min</li>
+            <li>Locations: none</li>
+        </ul>
+    </li>
+    <li>Marketing Event
+        <ul>
+            <li>Marketing Events: none</li>
+        </ul>
+    </li>
+    <li>Metafields

Review Comment:
   Some resources are mentioned here but cannot be chosen on the Processor itself (like `Metafields`)



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters" +
+                " can be found in additional details.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    static final PropertyDescriptor STORE_DOMAIN = new PropertyDescriptor.Builder()
+            .name("store-domain")
+            .displayName("Store Domain")
+            .description("The domain of the Shopify store, e.g. nifistore.myshopify.com")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Access Token")
+            .description("Access Token to authenticate requests")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The Shopify REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor OBJECT_CATEGORY = new PropertyDescriptor.Builder()
+            .name("object-category")
+            .displayName("Object Category")
+            .description("Shopify object category")
+            .required(true)
+            .allowableValues(ResourceType.class)
+            .build();
+
+    static final PropertyDescriptor RESULT_LIMIT = new PropertyDescriptor.Builder()
+            .name("result-limit")
+            .displayName("Result Limit")
+            .description("The maximum number of results to request for each invocation of the Processor")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.createLongValidator(1, 100, true))
+            .build();
+
+    static final PropertyDescriptor IS_INCREMENTAL = new PropertyDescriptor.Builder()
+            .name("is-incremental")
+            .displayName("Incremental Loading")
+            .description("The processor can incrementally load the queried objects so that each object is queried exactly once." +
+                    " For each query, the processor queries objects which were created or modified after the previous run time" +
+                    " but before the current time.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    static final PropertyDescriptor INCREMENTAL_DELAY = new PropertyDescriptor.Builder()
+            .name("incremental-delay")
+            .displayName("Incremental Delay")
+            .description("The ending timestamp of the time window will be adjusted earlier by the amount configured in this property." +
+                    " For example, with a property value of 10 seconds, an ending timestamp of 12:30:45 would be changed to 12:30:35.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    static final PropertyDescriptor INITIAL_INCREMENTAL_FILTER = new PropertyDescriptor.Builder()
+            .name("initial-incremental-filter")
+            .displayName("Initial Incremental Start Time")
+            .description("This property specifies the start time as Epoch Time that the processor applies when running the first request.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Controller service for HTTP client operations")
+            .required(true)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getPropertyDisplayName())
+                            .description(resourceType.getPropertyDescription())
+                            .required(true)
+                            .dependsOn(OBJECT_CATEGORY, resourceType.getValue())
+                            .allowableValues(resourceType.getResourcesAsAllowableValues())
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                STORE_DOMAIN,
+                ACCESS_TOKEN,
+                API_VERSION,
+                OBJECT_CATEGORY
+        ));
+        propertyDescriptors.addAll(resourceDescriptors);
+        propertyDescriptors.addAll(Arrays.asList(
+                RESULT_LIMIT,
+                IS_INCREMENTAL,
+                INCREMENTAL_DELAY,
+                INITIAL_INCREMENTAL_FILTER,
+                WEB_CLIENT_PROVIDER)
+        );
+        return Collections.unmodifiableList(propertyDescriptors);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Pattern CURSOR_PATTERN = Pattern.compile("<([^<]*)>; rel=\"next\"");
+
+    private volatile ShopifyRestService shopifyRestService;
+    private volatile ShopifyResource shopifyResource;
+    private volatile String resourceName;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final WebClientServiceProvider webClientServiceProvider =
+                context.getProperty(WEB_CLIENT_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        final WebClientService webClientService = webClientServiceProvider.getWebClientService();
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder();
+
+        final String apiVersion = context.getProperty(API_VERSION).getValue();
+        final String baseUrl = context.getProperty(STORE_DOMAIN).getValue();
+        final String accessToken = context.getProperty(ACCESS_TOKEN).getValue();
+
+        final String category = context.getProperty(OBJECT_CATEGORY).getValue();
+        final ResourceType resourceType = ResourceType.valueOf(category);
+        resourceName = context.getProperty(propertyMap.get(resourceType)).getValue();
+        final String limit = context.getProperty(RESULT_LIMIT).getValue();
+
+        shopifyResource = resourceType.getResource(resourceName);
+
+        shopifyRestService = getShopifyRestService(webClientService, uriBuilder, apiVersion, baseUrl, accessToken, resourceName,
+                limit, shopifyResource.getIncrementalLoadingParameter());
+    }
+
+    ShopifyRestService getShopifyRestService(final WebClientService webClientService, final HttpUriBuilder uriBuilder,
+                                             final String apiVersion, final String baseUrl, final String accessToken, final String resourceName,
+                                             final String limit, final IncrementalLoadingParameter incrementalLoadingParameter) {
+        return new ShopifyRestService(
+                webClientService,
+                uriBuilder,
+                apiVersion,
+                baseUrl,
+                accessToken,
+                resourceName,
+                limit,
+                incrementalLoadingParameter
+        );
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final StateMap state = getState(context);
+        final boolean isIncremental = context.getProperty(IS_INCREMENTAL).asBoolean();
+        final String initialStartTime = context.getProperty(INITIAL_INCREMENTAL_FILTER).getValue();
+        final Long incrDelayMs = context.getProperty(INCREMENTAL_DELAY).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        String lastExecutionTime = state.get(resourceName);
+        if (lastExecutionTime == null && initialStartTime != null) {
+            lastExecutionTime = initialStartTime;
+        }
+
+        Instant now = getCurrentExecutionTime();
+        if (incrDelayMs != null) {
+            now = now.minus(incrDelayMs, ChronoUnit.MILLIS);
+        }
+        final String currentExecutionTime = now.toString();
+
+        String cursor = null;
+        do {
+            final AtomicInteger objectCountHolder = new AtomicInteger();
+            try (HttpResponseEntity response = shopifyRestService.getShopifyObjects(isIncremental, lastExecutionTime, currentExecutionTime, cursor)) {
+                cursor = getPageCursor(response);
+                if (response.statusCode() == HttpResponseStatus.OK.getCode()) {
+                    FlowFile flowFile = session.create();
+                    flowFile = session.write(flowFile, parseHttpResponse(response, objectCountHolder));
+                    if (objectCountHolder.get() > 0) {
+                        session.transfer(flowFile, REL_SUCCESS);
+                        if (cursor == null && isIncremental) {

Review Comment:
   ```suggestion
                           if (cursor == null && isIncremental && shopifyResource.isPaginationSupported()) {
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters" +
+                " can be found in additional details.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    static final PropertyDescriptor STORE_DOMAIN = new PropertyDescriptor.Builder()
+            .name("store-domain")
+            .displayName("Store Domain")
+            .description("The domain of the Shopify store, e.g. nifistore.myshopify.com")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Access Token")
+            .description("Access Token to authenticate requests")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The Shopify REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor OBJECT_CATEGORY = new PropertyDescriptor.Builder()
+            .name("object-category")
+            .displayName("Object Category")
+            .description("Shopify object category")
+            .required(true)
+            .allowableValues(ResourceType.class)
+            .build();
+
+    static final PropertyDescriptor RESULT_LIMIT = new PropertyDescriptor.Builder()
+            .name("result-limit")
+            .displayName("Result Limit")
+            .description("The maximum number of results to request for each invocation of the Processor")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.createLongValidator(1, 100, true))
+            .build();
+
+    static final PropertyDescriptor IS_INCREMENTAL = new PropertyDescriptor.Builder()
+            .name("is-incremental")
+            .displayName("Incremental Loading")
+            .description("The processor can incrementally load the queried objects so that each object is queried exactly once." +
+                    " For each query, the processor queries objects which were created or modified after the previous run time" +
+                    " but before the current time.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    static final PropertyDescriptor INCREMENTAL_DELAY = new PropertyDescriptor.Builder()
+            .name("incremental-delay")
+            .displayName("Incremental Delay")
+            .description("The ending timestamp of the time window will be adjusted earlier by the amount configured in this property." +
+                    " For example, with a property value of 10 seconds, an ending timestamp of 12:30:45 would be changed to 12:30:35.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    static final PropertyDescriptor INITIAL_INCREMENTAL_FILTER = new PropertyDescriptor.Builder()
+            .name("initial-incremental-filter")
+            .displayName("Initial Incremental Start Time")
+            .description("This property specifies the start time as Epoch Time that the processor applies when running the first request.")
+            .required(false)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .dependsOn(IS_INCREMENTAL, "true")
+            .build();
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Controller service for HTTP client operations")
+            .required(true)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getPropertyDisplayName())
+                            .description(resourceType.getPropertyDescription())
+                            .required(true)
+                            .dependsOn(OBJECT_CATEGORY, resourceType.getValue())
+                            .allowableValues(resourceType.getResourcesAsAllowableValues())
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                STORE_DOMAIN,
+                ACCESS_TOKEN,
+                API_VERSION,
+                OBJECT_CATEGORY
+        ));
+        propertyDescriptors.addAll(resourceDescriptors);
+        propertyDescriptors.addAll(Arrays.asList(
+                RESULT_LIMIT,
+                IS_INCREMENTAL,
+                INCREMENTAL_DELAY,
+                INITIAL_INCREMENTAL_FILTER,
+                WEB_CLIENT_PROVIDER)
+        );
+        return Collections.unmodifiableList(propertyDescriptors);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Pattern CURSOR_PATTERN = Pattern.compile("<([^<]*)>; rel=\"next\"");
+
+    private volatile ShopifyRestService shopifyRestService;
+    private volatile ShopifyResource shopifyResource;
+    private volatile String resourceName;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final WebClientServiceProvider webClientServiceProvider =
+                context.getProperty(WEB_CLIENT_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        final WebClientService webClientService = webClientServiceProvider.getWebClientService();
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder();
+
+        final String apiVersion = context.getProperty(API_VERSION).getValue();
+        final String baseUrl = context.getProperty(STORE_DOMAIN).getValue();
+        final String accessToken = context.getProperty(ACCESS_TOKEN).getValue();
+
+        final String category = context.getProperty(OBJECT_CATEGORY).getValue();
+        final ResourceType resourceType = ResourceType.valueOf(category);
+        resourceName = context.getProperty(propertyMap.get(resourceType)).getValue();
+        final String limit = context.getProperty(RESULT_LIMIT).getValue();
+
+        shopifyResource = resourceType.getResource(resourceName);
+
+        shopifyRestService = getShopifyRestService(webClientService, uriBuilder, apiVersion, baseUrl, accessToken, resourceName,
+                limit, shopifyResource.getIncrementalLoadingParameter());
+    }
+
+    ShopifyRestService getShopifyRestService(final WebClientService webClientService, final HttpUriBuilder uriBuilder,
+                                             final String apiVersion, final String baseUrl, final String accessToken, final String resourceName,
+                                             final String limit, final IncrementalLoadingParameter incrementalLoadingParameter) {
+        return new ShopifyRestService(
+                webClientService,
+                uriBuilder,
+                apiVersion,
+                baseUrl,
+                accessToken,
+                resourceName,
+                limit,
+                incrementalLoadingParameter
+        );
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final StateMap state = getState(context);
+        final boolean isIncremental = context.getProperty(IS_INCREMENTAL).asBoolean();
+        final String initialStartTime = context.getProperty(INITIAL_INCREMENTAL_FILTER).getValue();
+        final Long incrDelayMs = context.getProperty(INCREMENTAL_DELAY).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        String lastExecutionTime = state.get(resourceName);
+        if (lastExecutionTime == null && initialStartTime != null) {
+            lastExecutionTime = initialStartTime;
+        }
+
+        Instant now = getCurrentExecutionTime();
+        if (incrDelayMs != null) {
+            now = now.minus(incrDelayMs, ChronoUnit.MILLIS);
+        }
+        final String currentExecutionTime = now.toString();
+
+        String cursor = null;
+        do {
+            final AtomicInteger objectCountHolder = new AtomicInteger();
+            try (HttpResponseEntity response = shopifyRestService.getShopifyObjects(isIncremental, lastExecutionTime, currentExecutionTime, cursor)) {
+                cursor = getPageCursor(response);
+                if (response.statusCode() == HttpResponseStatus.OK.getCode()) {
+                    FlowFile flowFile = session.create();
+                    flowFile = session.write(flowFile, parseHttpResponse(response, objectCountHolder));
+                    if (objectCountHolder.get() > 0) {
+                        session.transfer(flowFile, REL_SUCCESS);
+                        if (cursor == null && isIncremental) {
+                            final Map<String, String> stateMap = new HashMap<>(state.toMap());
+                            stateMap.put(resourceName, currentExecutionTime);
+                            updateState(context, stateMap);
+                        }
+                    } else {
+                        getLogger().debug("Empty response when requested Shopify resource: [{}]", resourceName);
+                        session.remove(flowFile);
+                    }
+                } else if (response.statusCode() >= 400) {
+                    if (response.statusCode() == TOO_MANY_REQUESTS) {
+                        context.yield();
+                        throw new ProcessException(String.format(
+                                "Rate limit exceeded, yielding before retrying request. HTTP %d error for requested URI [%s]",
+                                response.statusCode(), resourceName));
+                    } else {
+                        context.yield();
+                        getLogger().warn("HTTP {} error for requested Shopify resource [{}]", response.statusCode(),
+                                resourceName);
+                    }
+                }

Review Comment:
   Shouldn't we at least log something if the response code is neither 200 nor 400?



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

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

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


[GitHub] [nifi] krisztina-zsihovszki commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.nifi.components.AllowableValue;
+
+public class ResourceDirectory {
+
+    private ResourceDirectory() {
+    }
+
+    private static final Map<ResourceType, List<ShopifyResource>> resourceMap;
+
+    static {
+        resourceMap = new EnumMap<>(ResourceType.class);
+        resourceMap.put(ResourceType.CUSTOMERS, getCustomerResources());
+        resourceMap.put(ResourceType.DISCOUNTS, getDiscountResources());
+        resourceMap.put(ResourceType.INVENTORY, getInventoryResources());
+        resourceMap.put(ResourceType.ONLINE_STORE, getOnlineStoreResources());
+        resourceMap.put(ResourceType.ORDERS, getOrderResources());
+        resourceMap.put(ResourceType.PRODUCT, getProductResources());
+        resourceMap.put(ResourceType.SALES_CHANNELS, getSalesChannelResources());
+        resourceMap.put(ResourceType.STORE_PROPERTIES, getStorePropertyResources());
+    }
+
+    private static List<ShopifyResource> getCustomerResources() {
+        final ShopifyResource customer = ShopifyResource.newInstance(
+                "customers",
+                "Customers",
+                "The Customer resource stores information about a shop's customers, such as their contact details," +
+                        " their order history, and whether they've agreed to receive email marketing.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource customerSavedSearch = ShopifyResource.newInstance(
+                "customer_saved_searches",
+                "Customer Saved Searches",
+                "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(customer, customerSavedSearch));
+    }
+
+    private static List<ShopifyResource> getDiscountResources() {
+        final ShopifyResource priceRule = ShopifyResource.newInstance(
+                "price_rules",
+                "Price Rules",
+                "The PriceRule resource can be used to get discounts using conditions",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.singletonList(priceRule);
+    }
+
+    private static List<ShopifyResource> getInventoryResources() {
+        final ShopifyResource location = ShopifyResource.newInstance(
+                "locations",
+                "Locations",
+                "A location represents a geographical location where your stores, pop-up stores, headquarters, and warehouses exist.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.singletonList(location);
+    }
+
+    private static List<ShopifyResource> getOnlineStoreResources() {
+        final ShopifyResource blog = ShopifyResource.newInstance(
+                "blogs",
+                "Blogs",
+                "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource comment = ShopifyResource.newInstance(
+                "comments",
+                "Comments",
+                "A comment is a reader's response to an article in a blog.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource page = ShopifyResource.newInstance(
+                "pages",
+                "Pages",
+                "Shopify stores come with a tool for creating basic HTML web pages.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource redirect = ShopifyResource.newInstance(
+                "redirects",
+                "Redirects",
+                "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource scriptTag = ShopifyResource.newInstance(
+                "script_tags",
+                "Script Tags",
+                "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource theme = ShopifyResource.newInstance(
+                "themes",
+                "Themes",
+                "A theme controls the look and feel of a Shopify online store.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(blog, comment, page, redirect, scriptTag, theme));
+    }
+
+    private static List<ShopifyResource> getOrderResources() {
+        final ShopifyResource abandonedCheckouts = ShopifyResource.newInstance(
+                "checkouts",
+                "Abandoned Checkouts",
+                "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource draftOrders = ShopifyResource.newInstance(
+                "draft_orders",
+                "Draft Orders",
+                "Merchants can use draft orders to create orders on behalf of their customers.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource orders = ShopifyResource.newInstance(
+                "orders",
+                "Orders",
+                "An order is a customer's request to purchase one or more products from a shop.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(abandonedCheckouts, draftOrders, orders));
+    }
+
+    private static List<ShopifyResource> getProductResources() {
+        final ShopifyResource collect = ShopifyResource.newInstance(
+                "collects",
+                "Collects",
+                "Collects are meant for managing the relationship between products and custom collections.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource customCollection = ShopifyResource.newInstance(
+                "custom_collections",
+                "Custom Collections",
+                "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource product = ShopifyResource.newInstance(
+                "products",
+                "Products",
+                "Get products in a merchant's store ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource smartCollection = ShopifyResource.newInstance(
+                "smart_collections",
+                "Smart Collections",
+                "A smart collection is a grouping of products defined by rules that are set by the merchant.",

Review Comment:
   The name of resource is sometimes written with lower case (e.g. "smart collection"), sometimes with camel case (e.g. "CollectionListing") in the description . 



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

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

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


[GitHub] [nifi] tpalfy commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty

Review Comment:
   `@TriggerWhenEmpty` shouldn't be necessary as input is forbidden.
   



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceType.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.shopify.model;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.DescribedValue;
+
+public enum ResourceType implements DescribedValue {

Review Comment:
   ```suggestion
   public enum ResourceType implements DescribedValue {
   
       CUSTOMERS("Customers", "Query a Customer resource", "Customer Resource", "Customer resource to query", ResourceDirectory.CUSTOMER_RESOURCES),
       DISCOUNTS("Discounts", "Query a Discount resource", "Discount Resource", "Discount resource to query", ResourceDirectory.DISCOUNT_RESOURCES),
       INVENTORY("Inventory", "Query an Inventory resource", "Inventory Resource", "Inventory resource to query", ResourceDirectory.INVENTORY_RESOURCES),
       ONLINE_STORE("Online Store", "Query an Online Store resource", "Online Store Resource", "Online Store resource to query", ResourceDirectory.ONLINE_STORE_RESOURCES),
       ORDERS("Orders", "Query an Order resource", "Order Resource", "Order resource to query", ResourceDirectory.ORDER_RESOURCES),
       PRODUCT("Products", "Query a Product resource", "Product Resource", "Product resource to query", ResourceDirectory.PRODUCT_RESOURCES),
       SALES_CHANNELS("Sales Channels", "Query a Sales Channel resource", "Sales Channel Resource", "Sales Channel resource to query", ResourceDirectory.SALES_CHANNEL_RESOURCES),
       STORE_PROPERTIES("Store Properties", "Query a Store Property resource", "Store Property Resource", "Store Property resource to query", ResourceDirectory.STORE_PROPERTY_RESOURCES);
   
       private final String allowableValueDisplayName;
       private final String allowableValueDescription;
       private final String propertyDisplayName;
       private final String propertyDescription;
       private List<ShopifyResource> resources;
   
       ResourceType(
               final String allowableValueDisplayName,
               final String allowableValueDescription,
               String propertyDisplayName, String propertyDescription, List<ShopifyResource> resources
       ) {
           this.allowableValueDisplayName = allowableValueDisplayName;
           this.allowableValueDescription = allowableValueDescription;
           this.propertyDisplayName = propertyDisplayName;
           this.propertyDescription = propertyDescription;
           this.resources = resources;
       }
   
       @Override
       public String getValue() {
           return name();
       }
   
       @Override
       public String getDisplayName() {
           return allowableValueDisplayName;
       }
   
       @Override
       public String getDescription() {
           return allowableValueDescription;
       }
   
       public String getPropertyDescription() {
           return propertyDescription;
       }
   
       public String getPropertyDisplayName() {
           return propertyDisplayName;
       }
   
       public List<ShopifyResource> getResources() {
           return resources;
       }
   
       public ShopifyResource getResource(final String value) {
           return getResources().stream()
                   .filter(s -> s.getValue().equals(value))
                   .findFirst()
                   .get();
       }
   
       public AllowableValue[] getResourcesAsAllowableValues() {
           return getResources().stream().map(ShopifyResource::getAllowableValue).toArray(AllowableValue[]::new);
       }
   }
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.nifi.components.AllowableValue;
+
+public class ResourceDirectory {
+
+    private ResourceDirectory() {
+    }
+
+    private static final Map<ResourceType, List<ShopifyResource>> resourceMap;
+
+    static {
+        resourceMap = new EnumMap<>(ResourceType.class);
+        resourceMap.put(ResourceType.CUSTOMERS, getCustomerResources());
+        resourceMap.put(ResourceType.DISCOUNTS, getDiscountResources());
+        resourceMap.put(ResourceType.INVENTORY, getInventoryResources());
+        resourceMap.put(ResourceType.ONLINE_STORE, getOnlineStoreResources());
+        resourceMap.put(ResourceType.ORDERS, getOrderResources());
+        resourceMap.put(ResourceType.PRODUCT, getProductResources());
+        resourceMap.put(ResourceType.SALES_CHANNELS, getSalesChannelResources());
+        resourceMap.put(ResourceType.STORE_PROPERTIES, getStorePropertyResources());
+    }
+
+    private static List<ShopifyResource> getCustomerResources() {
+        final ShopifyResource customer = ShopifyResource.newInstance(
+                "customers",
+                "Customers",
+                "The Customer resource stores information about a shop's customers, such as their contact details," +
+                        " their order history, and whether they've agreed to receive email marketing.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource customerSavedSearch = ShopifyResource.newInstance(
+                "customer_saved_searches",
+                "Customer Saved Searches",
+                "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(customer, customerSavedSearch));
+    }
+
+    private static List<ShopifyResource> getDiscountResources() {
+        final ShopifyResource priceRule = ShopifyResource.newInstance(
+                "price_rules",
+                "Price Rules",
+                "The PriceRule resource can be used to get discounts using conditions",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.singletonList(priceRule);
+    }
+
+    private static List<ShopifyResource> getInventoryResources() {
+        final ShopifyResource location = ShopifyResource.newInstance(
+                "locations",
+                "Locations",
+                "A location represents a geographical location where your stores, pop-up stores, headquarters, and warehouses exist.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.singletonList(location);
+    }
+
+    private static List<ShopifyResource> getOnlineStoreResources() {
+        final ShopifyResource blog = ShopifyResource.newInstance(
+                "blogs",
+                "Blogs",
+                "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource comment = ShopifyResource.newInstance(
+                "comments",
+                "Comments",
+                "A comment is a reader's response to an article in a blog.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource page = ShopifyResource.newInstance(
+                "pages",
+                "Pages",
+                "Shopify stores come with a tool for creating basic HTML web pages.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource redirect = ShopifyResource.newInstance(
+                "redirects",
+                "Redirects",
+                "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource scriptTag = ShopifyResource.newInstance(
+                "script_tags",
+                "Script Tags",
+                "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource theme = ShopifyResource.newInstance(
+                "themes",
+                "Themes",
+                "A theme controls the look and feel of a Shopify online store.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(blog, comment, page, redirect, scriptTag, theme));
+    }
+
+    private static List<ShopifyResource> getOrderResources() {
+        final ShopifyResource abandonedCheckouts = ShopifyResource.newInstance(
+                "checkouts",
+                "Abandoned Checkouts",
+                "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource draftOrders = ShopifyResource.newInstance(
+                "draft_orders",
+                "Draft Orders",
+                "Merchants can use draft orders to create orders on behalf of their customers.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource orders = ShopifyResource.newInstance(
+                "orders",
+                "Orders",
+                "An order is a customer's request to purchase one or more products from a shop.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(abandonedCheckouts, draftOrders, orders));
+    }
+
+    private static List<ShopifyResource> getProductResources() {
+        final ShopifyResource collect = ShopifyResource.newInstance(
+                "collects",
+                "Collects",
+                "Collects are meant for managing the relationship between products and custom collections.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource customCollection = ShopifyResource.newInstance(
+                "custom_collections",
+                "Custom Collections",
+                "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource product = ShopifyResource.newInstance(
+                "products",
+                "Products",
+                "Get products in a merchant's store ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource smartCollection = ShopifyResource.newInstance(
+                "smart_collections",
+                "Smart Collections",
+                "A smart collection is a grouping of products defined by rules that are set by the merchant.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(collect, customCollection, product, smartCollection));
+    }
+
+    private static List<ShopifyResource> getSalesChannelResources() {
+        final ShopifyResource collectionListing = ShopifyResource.newInstance(
+                "collection_listings",
+                "Collection Listings",
+                "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource productListing = ShopifyResource.newInstance(
+                "product_listings",
+                "Product Listings",
+                "A ProductListing resource represents a Product which is available to your sales channel.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(collectionListing, productListing));
+    }
+
+    private static List<ShopifyResource> getStorePropertyResources() {
+        final ShopifyResource country = ShopifyResource.newInstance(
+                "countries",
+                "Countries",
+                "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource currency = ShopifyResource.newInstance(
+                "currencies",
+                "Currencies",
+                "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource policy = ShopifyResource.newInstance(
+                "policies",
+                "Policies",
+                "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource shippingZone = ShopifyResource.newInstance(
+                "shipping_zones",
+                "Shipping Zones",
+                "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource shop = ShopifyResource.newInstance(
+                "shop",
+                "Shop",
+                "The Shop resource is a collection of general business and store management settings and information about the store.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(country, currency, policy, shippingZone, shop));
+    }
+
+    public static AllowableValue[] getCategories() {
+        return resourceMap.keySet().stream().map(ResourceType::getAllowableValue).toArray(AllowableValue[]::new);
+    }
+
+    public static List<ShopifyResource> getResources(final ResourceType key) {
+        return resourceMap.get(key);
+    }
+
+    public static AllowableValue[] getResourcesAsAllowableValues(final ResourceType key) {
+        return getResources(key).stream().map(ShopifyResource::getAllowableValue).toArray(AllowableValue[]::new);
+    }
+
+    public static ShopifyResource getResourceTypeDto(final ResourceType key, final String value) {
+        return getResources(key).stream()
+                .filter(s -> s.getValue().equals(value))
+                .findFirst()
+                .get();
+    }

Review Comment:
   ```suggestion
   public static final List<ShopifyResource> CUSTOMER_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "customers",
               "Customers",
               "The Customer resource stores information about a shop's customers, such as their contact details," +
                       " their order history, and whether they've agreed to receive email marketing.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "customer_saved_searches",
               "Customer Saved Searches",
               "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
               IncrementalLoadingParameter.NONE
       )));
   
       public static final List<ShopifyResource> DISCOUNT_RESOURCES = Collections.singletonList(ShopifyResource.newInstance(
               "price_rules",
               "Price Rules",
               "The PriceRule resource can be used to get discounts using conditions",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ));
   
       public static final List<ShopifyResource> INVENTORY_RESOURCES = Collections.singletonList(ShopifyResource.newInstance(
               "locations",
               "Locations",
               "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
               IncrementalLoadingParameter.NONE
       ));
   
       public static final List<ShopifyResource> ONLINE_STORE_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "blogs",
               "Blogs",
               "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "comments",
               "Comments",
               "A comment is a reader's response to an article in a blog.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "pages",
               "Pages",
               "Shopify stores come with a tool for creating basic HTML web pages.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "redirects",
               "Redirects",
               "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "script_tags",
               "Script Tags",
               "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "themes",
               "Themes",
               "A theme controls the look and feel of a Shopify online store.",
               IncrementalLoadingParameter.NONE
       )));
   
       public static final List<ShopifyResource> ORDER_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "checkouts",
               "Abandoned Checkouts",
               "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "draft_orders",
               "Draft Orders",
               "Merchants can use draft orders to create orders on behalf of their customers.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "orders",
               "Orders",
               "An order is a customer's request to purchase one or more products from a shop.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> PRODUCT_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "collects",
               "Collects",
               "Collects are meant for managing the relationship between products and custom collections.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "custom_collections",
               "Custom Collections",
               "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "products",
               "Products",
               "Get products in a merchant's store ",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "smart_collections",
               "Smart Collections",
               "A smart collection is a grouping of products defined by rules that are set by the merchant.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "collection_listings",
               "Collection Listings",
               "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "product_listings",
               "Product Listings",
               "A ProductListing resource represents a Product which is available to your sales channel.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "countries",
               "Countries",
               "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "currencies",
               "Currencies",
               "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "policies",
               "Policies",
               "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "shipping_zones",
               "Shipping Zones",
               "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "shop",
               "Shop",
               "The Shop resource is a collection of general business and store management settings and information about the store.",
               IncrementalLoadingParameter.NONE
       )));
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")
+            .description("Shopify resource type")
+            .required(true)
+            .allowableValues(ResourceDirectory.getCategories())
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getDisplayName())
+                            .description(resourceType.getDescription())
+                            .required(true)
+                            .dependsOn(RESOURCE_TYPE, resourceType.getValue())
+                            .allowableValues(ResourceDirectory.getResourcesAsAllowableValues(resourceType))
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                WEB_CLIENT_PROVIDER,
+                API_URL,
+                ACCESS_TOKEN,
+                API_VERSION,
+                RESOURCE_TYPE
+        ));
+        propertyDescriptors.addAll(resourceDescriptors);
+        return Collections.unmodifiableList(propertyDescriptors);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+    private static final int TOO_MANY_REQUESTS = 429;
+
+    private volatile ShopifyRestService shopifyRestService;
+    private volatile ShopifyResource shopifyResource;
+    private volatile String resourceName;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final WebClientServiceProvider webClientServiceProvider =
+                context.getProperty(WEB_CLIENT_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        final WebClientService webClientService = webClientServiceProvider.getWebClientService();
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder();
+
+        final String apiVersion = context.getProperty(API_VERSION).getValue();
+        final String baseUrl = context.getProperty(API_URL).getValue();
+        final String accessToken = context.getProperty(ACCESS_TOKEN).getValue();
+
+        final String category = context.getProperty(RESOURCE_TYPE).getValue();
+        final ResourceType resourceType = ResourceType.valueOf(category);
+        resourceName = context.getProperty(propertyMap.get(resourceType)).getValue();
+
+        shopifyResource = ResourceDirectory.getResourceTypeDto(resourceType, resourceName);

Review Comment:
   ```suggestion
           shopifyResource = resourceType.getResource(resourceName);
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")
+            .description("Shopify resource type")
+            .required(true)
+            .allowableValues(ResourceDirectory.getCategories())
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getDisplayName())
+                            .description(resourceType.getDescription())

Review Comment:
   ```suggestion
                               .displayName(resourceType.getPropertyDisplayName())
                               .description(resourceType.getPropertyDescription())
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")
+            .description("Shopify resource type")
+            .required(true)
+            .allowableValues(ResourceDirectory.getCategories())
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getDisplayName())
+                            .description(resourceType.getDescription())
+                            .required(true)
+                            .dependsOn(RESOURCE_TYPE, resourceType.getValue())
+                            .allowableValues(ResourceDirectory.getResourcesAsAllowableValues(resourceType))

Review Comment:
   ```suggestion
                               .allowableValues(resourceType.getResourcesAsAllowableValues())
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")
+            .description("Shopify resource type")
+            .required(true)
+            .allowableValues(ResourceDirectory.getCategories())

Review Comment:
   ```suggestion
               .allowableValues(ResourceType.values())
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.nifi.components.AllowableValue;
+
+public class ResourceDirectory {

Review Comment:
   ```suggestion
   public class ResourceDirectory {
   
       public static final List<ShopifyResource> CUSTOMER_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "customers",
               "Customers",
               "The Customer resource stores information about a shop's customers, such as their contact details," +
                       " their order history, and whether they've agreed to receive email marketing.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "customer_saved_searches",
               "Customer Saved Searches",
               "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
               IncrementalLoadingParameter.NONE
       )));
   
       public static final List<ShopifyResource> DISCOUNT_RESOURCES = Collections.singletonList(ShopifyResource.newInstance(
               "price_rules",
               "Price Rules",
               "The PriceRule resource can be used to get discounts using conditions",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ));
   
       public static final List<ShopifyResource> INVENTORY_RESOURCES = Collections.singletonList(ShopifyResource.newInstance(
               "locations",
               "Locations",
               "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
               IncrementalLoadingParameter.NONE
       ));
   
       public static final List<ShopifyResource> ONLINE_STORE_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "blogs",
               "Blogs",
               "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "comments",
               "Comments",
               "A comment is a reader's response to an article in a blog.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "pages",
               "Pages",
               "Shopify stores come with a tool for creating basic HTML web pages.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "redirects",
               "Redirects",
               "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "script_tags",
               "Script Tags",
               "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "themes",
               "Themes",
               "A theme controls the look and feel of a Shopify online store.",
               IncrementalLoadingParameter.NONE
       )));
   
       public static final List<ShopifyResource> ORDER_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "checkouts",
               "Abandoned Checkouts",
               "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "draft_orders",
               "Draft Orders",
               "Merchants can use draft orders to create orders on behalf of their customers.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "orders",
               "Orders",
               "An order is a customer's request to purchase one or more products from a shop.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> PRODUCT_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "collects",
               "Collects",
               "Collects are meant for managing the relationship between products and custom collections.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "custom_collections",
               "Custom Collections",
               "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "products",
               "Products",
               "Get products in a merchant's store ",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "smart_collections",
               "Smart Collections",
               "A smart collection is a grouping of products defined by rules that are set by the merchant.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "collection_listings",
               "Collection Listings",
               "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "product_listings",
               "Product Listings",
               "A ProductListing resource represents a Product which is available to your sales channel.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "countries",
               "Countries",
               "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "currencies",
               "Currencies",
               "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "policies",
               "Policies",
               "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "shipping_zones",
               "Shipping Zones",
               "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "shop",
               "Shop",
               "The Shop resource is a collection of general business and store management settings and information about the store.",
               IncrementalLoadingParameter.NONE
       )));
   }
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.nifi.components.AllowableValue;
+
+public class ResourceDirectory {
+
+    private ResourceDirectory() {
+    }
+
+    private static final Map<ResourceType, List<ShopifyResource>> resourceMap;
+
+    static {
+        resourceMap = new EnumMap<>(ResourceType.class);
+        resourceMap.put(ResourceType.CUSTOMERS, getCustomerResources());
+        resourceMap.put(ResourceType.DISCOUNTS, getDiscountResources());
+        resourceMap.put(ResourceType.INVENTORY, getInventoryResources());
+        resourceMap.put(ResourceType.ONLINE_STORE, getOnlineStoreResources());
+        resourceMap.put(ResourceType.ORDERS, getOrderResources());
+        resourceMap.put(ResourceType.PRODUCT, getProductResources());
+        resourceMap.put(ResourceType.SALES_CHANNELS, getSalesChannelResources());
+        resourceMap.put(ResourceType.STORE_PROPERTIES, getStorePropertyResources());
+    }
+
+    private static List<ShopifyResource> getCustomerResources() {
+        final ShopifyResource customer = ShopifyResource.newInstance(
+                "customers",
+                "Customers",
+                "The Customer resource stores information about a shop's customers, such as their contact details," +
+                        " their order history, and whether they've agreed to receive email marketing.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource customerSavedSearch = ShopifyResource.newInstance(
+                "customer_saved_searches",
+                "Customer Saved Searches",
+                "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(customer, customerSavedSearch));
+    }
+
+    private static List<ShopifyResource> getDiscountResources() {
+        final ShopifyResource priceRule = ShopifyResource.newInstance(
+                "price_rules",
+                "Price Rules",
+                "The PriceRule resource can be used to get discounts using conditions",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.singletonList(priceRule);
+    }
+
+    private static List<ShopifyResource> getInventoryResources() {
+        final ShopifyResource location = ShopifyResource.newInstance(
+                "locations",
+                "Locations",
+                "A location represents a geographical location where your stores, pop-up stores, headquarters, and warehouses exist.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.singletonList(location);
+    }
+
+    private static List<ShopifyResource> getOnlineStoreResources() {
+        final ShopifyResource blog = ShopifyResource.newInstance(
+                "blogs",
+                "Blogs",
+                "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource comment = ShopifyResource.newInstance(
+                "comments",
+                "Comments",
+                "A comment is a reader's response to an article in a blog.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource page = ShopifyResource.newInstance(
+                "pages",
+                "Pages",
+                "Shopify stores come with a tool for creating basic HTML web pages.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource redirect = ShopifyResource.newInstance(
+                "redirects",
+                "Redirects",
+                "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource scriptTag = ShopifyResource.newInstance(
+                "script_tags",
+                "Script Tags",
+                "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource theme = ShopifyResource.newInstance(
+                "themes",
+                "Themes",
+                "A theme controls the look and feel of a Shopify online store.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(blog, comment, page, redirect, scriptTag, theme));
+    }
+
+    private static List<ShopifyResource> getOrderResources() {
+        final ShopifyResource abandonedCheckouts = ShopifyResource.newInstance(
+                "checkouts",
+                "Abandoned Checkouts",
+                "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource draftOrders = ShopifyResource.newInstance(
+                "draft_orders",
+                "Draft Orders",
+                "Merchants can use draft orders to create orders on behalf of their customers.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource orders = ShopifyResource.newInstance(
+                "orders",
+                "Orders",
+                "An order is a customer's request to purchase one or more products from a shop.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(abandonedCheckouts, draftOrders, orders));
+    }
+
+    private static List<ShopifyResource> getProductResources() {
+        final ShopifyResource collect = ShopifyResource.newInstance(
+                "collects",
+                "Collects",
+                "Collects are meant for managing the relationship between products and custom collections.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource customCollection = ShopifyResource.newInstance(
+                "custom_collections",
+                "Custom Collections",
+                "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource product = ShopifyResource.newInstance(
+                "products",
+                "Products",
+                "Get products in a merchant's store ",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource smartCollection = ShopifyResource.newInstance(
+                "smart_collections",
+                "Smart Collections",
+                "A smart collection is a grouping of products defined by rules that are set by the merchant.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(collect, customCollection, product, smartCollection));
+    }
+
+    private static List<ShopifyResource> getSalesChannelResources() {
+        final ShopifyResource collectionListing = ShopifyResource.newInstance(
+                "collection_listings",
+                "Collection Listings",
+                "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource productListing = ShopifyResource.newInstance(
+                "product_listings",
+                "Product Listings",
+                "A ProductListing resource represents a Product which is available to your sales channel.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        return Collections.unmodifiableList(Arrays.asList(collectionListing, productListing));
+    }
+
+    private static List<ShopifyResource> getStorePropertyResources() {
+        final ShopifyResource country = ShopifyResource.newInstance(
+                "countries",
+                "Countries",
+                "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource currency = ShopifyResource.newInstance(
+                "currencies",
+                "Currencies",
+                "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource policy = ShopifyResource.newInstance(
+                "policies",
+                "Policies",
+                "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
+                IncrementalLoadingParameter.NONE
+        );
+        final ShopifyResource shippingZone = ShopifyResource.newInstance(
+                "shipping_zones",
+                "Shipping Zones",
+                "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
+                IncrementalLoadingParameter.UPDATED_AT_MIN
+        );
+        final ShopifyResource shop = ShopifyResource.newInstance(
+                "shop",
+                "Shop",
+                "The Shop resource is a collection of general business and store management settings and information about the store.",
+                IncrementalLoadingParameter.NONE
+        );
+        return Collections.unmodifiableList(Arrays.asList(country, currency, policy, shippingZone, shop));
+    }
+
+    public static AllowableValue[] getCategories() {
+        return resourceMap.keySet().stream().map(ResourceType::getAllowableValue).toArray(AllowableValue[]::new);
+    }
+
+    public static List<ShopifyResource> getResources(final ResourceType key) {
+        return resourceMap.get(key);
+    }
+
+    public static AllowableValue[] getResourcesAsAllowableValues(final ResourceType key) {
+        return getResources(key).stream().map(ShopifyResource::getAllowableValue).toArray(AllowableValue[]::new);
+    }
+
+    public static ShopifyResource getResourceTypeDto(final ResourceType key, final String value) {
+        return getResources(key).stream()
+                .filter(s -> s.getValue().equals(value))
+                .findFirst()
+                .get();
+    }

Review Comment:
   ```suggestion
   public static final List<ShopifyResource> CUSTOMER_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "customers",
               "Customers",
               "The Customer resource stores information about a shop's customers, such as their contact details," +
                       " their order history, and whether they've agreed to receive email marketing.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "customer_saved_searches",
               "Customer Saved Searches",
               "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
               IncrementalLoadingParameter.NONE
       )));
   
       public static final List<ShopifyResource> DISCOUNT_RESOURCES = Collections.singletonList(ShopifyResource.newInstance(
               "price_rules",
               "Price Rules",
               "The PriceRule resource can be used to get discounts using conditions",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ));
   
       public static final List<ShopifyResource> INVENTORY_RESOURCES = Collections.singletonList(ShopifyResource.newInstance(
               "locations",
               "Locations",
               "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
               IncrementalLoadingParameter.NONE
       ));
   
       public static final List<ShopifyResource> ONLINE_STORE_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "blogs",
               "Blogs",
               "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "comments",
               "Comments",
               "A comment is a reader's response to an article in a blog.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "pages",
               "Pages",
               "Shopify stores come with a tool for creating basic HTML web pages.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "redirects",
               "Redirects",
               "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "script_tags",
               "Script Tags",
               "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "themes",
               "Themes",
               "A theme controls the look and feel of a Shopify online store.",
               IncrementalLoadingParameter.NONE
       )));
   
       public static final List<ShopifyResource> ORDER_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "checkouts",
               "Abandoned Checkouts",
               "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "draft_orders",
               "Draft Orders",
               "Merchants can use draft orders to create orders on behalf of their customers.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "orders",
               "Orders",
               "An order is a customer's request to purchase one or more products from a shop.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> PRODUCT_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "collects",
               "Collects",
               "Collects are meant for managing the relationship between products and custom collections.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "custom_collections",
               "Custom Collections",
               "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "products",
               "Products",
               "Get products in a merchant's store ",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "smart_collections",
               "Smart Collections",
               "A smart collection is a grouping of products defined by rules that are set by the merchant.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "collection_listings",
               "Collection Listings",
               "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "product_listings",
               "Product Listings",
               "A ProductListing resource represents a Product which is available to your sales channel.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       )));
   
       public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES = Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
               "countries",
               "Countries",
               "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "currencies",
               "Currencies",
               "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "policies",
               "Policies",
               "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
               IncrementalLoadingParameter.NONE
       ), ShopifyResource.newInstance(
               "shipping_zones",
               "Shipping Zones",
               "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
               IncrementalLoadingParameter.UPDATED_AT_MIN
       ), ShopifyResource.newInstance(
               "shop",
               "Shop",
               "The Shop resource is a collection of general business and store management settings and information about the store.",
               IncrementalLoadingParameter.NONE
       )));
   ```



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

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

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


[GitHub] [nifi] Lehel44 commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/resources/docs/org.apache.nifi.shopify.processors.GetShopify/additionalDetails.html:
##########
@@ -0,0 +1,148 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>GetShopify</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<h2>Setting Up a Custom App</h2>
+<p>
+    Follow the <a href="https://help.shopify.com/en/manual/apps/custom-apps">Shopify tutorial</a> to enable and create
+    private apps, set API Scopes and generate API tokens.
+</p>
+<h2>Incremental Loading</h2>
+<p>
+    Some resources can be processed incrementally by NiFi. This means that only resources created or modified after the
+    last run time of the processor are displayed. The processor state can be reset in the context menu. The following
+    list shows which date-time fields are incremented for which resources.
+<ul>
+    <li>Access
+        <ul>
+            <li>Access Scope: none</li>

Review Comment:
   I also synchronized it with ResourceType and removed the extra ones.



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

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

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


[GitHub] [nifi] Lehel44 commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class ResourceDirectory {
+
+    private static final boolean PAGINATION_SUPPORTED = true;
+    private static final boolean PAGINATION_NOT_SUPPORTED = false;
+
+    public static final List<ShopifyResource> CUSTOMER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "customers",
+                    "Customers",
+                    "The Customer resource stores information about a shop's customers, such as their contact details,"
+                            + " their order history, and whether they've agreed to receive email marketing.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "customer_saved_searches",
+                    "Customer Saved Searches",
+                    "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> DISCOUNT_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "price_rules",
+                    "Price Rules",
+                    "The PriceRule resource can be used to get discounts using conditions",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> INVENTORY_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "locations",
+                    "Locations",
+                    "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> ONLINE_STORE_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "blogs",
+                    "Blogs",
+                    "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "comments",
+                    "Comments",
+                    "A comment is a reader's response to an article in a blog.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "pages",
+                    "Pages",
+                    "Shopify stores come with a tool for creating basic HTML web pages.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "redirects",
+                    "Redirects",
+                    "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "script_tags",
+                    "Script Tags",
+                    "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "themes",
+                    "Themes",
+                    "A theme controls the look and feel of a Shopify online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            )));
+    public static final List<ShopifyResource> ORDER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "checkouts",
+                    "Abandoned Checkouts",
+                    "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "draft_orders",
+                    "Draft Orders",
+                    "Merchants can use draft orders to create orders on behalf of their customers.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "orders",
+                    "Orders",
+                    "An order is a customer's request to purchase one or more products from a shop.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> PRODUCT_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collects",
+                    "Collects",
+                    "Collects are meant for managing the relationship between products and custom collections.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "custom_collections",
+                    "Custom Collections",
+                    "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "products",
+                    "Products",
+                    "Get products in a merchant's store ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "smart_collections",
+                    "Smart Collections",
+                    "A smart collection is a grouping of products defined by rules that are set by the merchant.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collection_listings",
+                    "Collection Listings",
+                    "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "product_listings",
+                    "Product Listings",
+                    "A ProductListing resource represents a Product which is available to your sales channel.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "countries",
+                    "Countries",
+                    "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "currencies",
+                    "Currencies",
+                    "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "policies",
+                    "Policies",
+                    "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "shipping_zones",
+                    "Shipping Zones",
+                    "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(

Review Comment:
   It does not return a zip for me but the problem is that the Json parser expects that the endpoints contain an array of objects. The 'Shop' endpoint is special because there is only on object wrapped under the 'shop' field.
   
   "shop": {
           "id": 65529675995,
           "name": "nifi-test",
           "email": "asd@asd.com",
           ...
   }
   I think it's better to remove it for now. 



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

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

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


[GitHub] [nifi] Lehel44 commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class ResourceDirectory {
+
+    private static final boolean PAGINATION_SUPPORTED = true;
+    private static final boolean PAGINATION_NOT_SUPPORTED = false;
+
+    public static final List<ShopifyResource> CUSTOMER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "customers",
+                    "Customers",
+                    "The Customer resource stores information about a shop's customers, such as their contact details,"
+                            + " their order history, and whether they've agreed to receive email marketing.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "customer_saved_searches",
+                    "Customer Saved Searches",
+                    "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> DISCOUNT_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "price_rules",
+                    "Price Rules",
+                    "The PriceRule resource can be used to get discounts using conditions",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> INVENTORY_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "locations",
+                    "Locations",
+                    "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> ONLINE_STORE_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "blogs",
+                    "Blogs",
+                    "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "comments",
+                    "Comments",
+                    "A comment is a reader's response to an article in a blog.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "pages",
+                    "Pages",
+                    "Shopify stores come with a tool for creating basic HTML web pages.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "redirects",
+                    "Redirects",
+                    "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "script_tags",
+                    "Script Tags",
+                    "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "themes",
+                    "Themes",
+                    "A theme controls the look and feel of a Shopify online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            )));
+    public static final List<ShopifyResource> ORDER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "checkouts",
+                    "Abandoned Checkouts",
+                    "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "draft_orders",
+                    "Draft Orders",
+                    "Merchants can use draft orders to create orders on behalf of their customers.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "orders",
+                    "Orders",
+                    "An order is a customer's request to purchase one or more products from a shop.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> PRODUCT_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collects",
+                    "Collects",
+                    "Collects are meant for managing the relationship between products and custom collections.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "custom_collections",
+                    "Custom Collections",
+                    "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "products",
+                    "Products",
+                    "Get products in a merchant's store ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "smart_collections",
+                    "Smart Collections",
+                    "A smart collection is a grouping of products defined by rules that are set by the merchant.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collection_listings",
+                    "Collection Listings",
+                    "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "product_listings",
+                    "Product Listings",
+                    "A ProductListing resource represents a Product which is available to your sales channel.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "countries",
+                    "Countries",
+                    "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "currencies",
+                    "Currencies",
+                    "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "policies",
+                    "Policies",
+                    "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "shipping_zones",
+                    "Shipping Zones",
+                    "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(

Review Comment:
   It does not return a zip for me but the problem is that the Json parser expects that the endpoints contain an array of objects. The 'Shop' endpoint is special because there is only on object wrapped under the 'shop' field.
   
   "shop": {
           "id": 65529675995,
           "name": "nifi-test",
           "email": "lehelb@cloudera.com",
           ...
   }
   I think it's better to remove it for now. 



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

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

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


[GitHub] [nifi] asfgit closed pull request #6303: NIFI-10213: Create GetShopify processor

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #6303: NIFI-10213: Create GetShopify processor
URL: https://github.com/apache/nifi/pull/6303


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

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

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


[GitHub] [nifi] tpalfy commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceType.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.shopify.model;
+
+import java.util.List;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.DescribedValue;
+
+public enum ResourceType implements DescribedValue {
+
+    CUSTOMERS(
+            "Customers",
+            "Query a Customer resource",
+            "Customer Categories",

Review Comment:
   I think the `... Categories` label is misleading and inadequate. This is a label for a property through which the user is selecting _one_ resource (out of many) that they want to query.
   ```suggestion
               "Customer Resource",
   
   ```



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

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

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


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/pom.xml:
##########
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-shopify-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-shopify-processors</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+        </dependency>

Review Comment:
   This dependency is not used and can be removed.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/pom.xml:
##########
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-shopify-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-shopify-processors</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <!-- Test dependencies -->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-service</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service</artifactId>
+            <scope>test</scope>
+        </dependency>

Review Comment:
   This dependency is not used and can be removed.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")

Review Comment:
   This property expects only the "hostname" part of the URL, not the whole API URL.
   E.g. `nifistore.myshopify.com` instead of `https://nifistore.myshopify.com/admin/api`.
   So I think `Store Domain` / `The domain of the Shopify store` would be a better name / description for it.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")
+            .description("Shopify resource type")
+            .required(true)
+            .allowableValues(ResourceDirectory.getCategories())
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getDisplayName())
+                            .description(resourceType.getDescription())
+                            .required(true)
+                            .dependsOn(RESOURCE_TYPE, resourceType.getValue())
+                            .allowableValues(ResourceDirectory.getResourcesAsAllowableValues(resourceType))
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                WEB_CLIENT_PROVIDER,

Review Comment:
   Similar to `GetHubSpot`, this property can go to the bottom of the property list.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-nar/pom.xml:
##########
@@ -0,0 +1,46 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-shopify-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-shopify-nar</artifactId>
+
+    <packaging>nar</packaging>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-shopify-processors</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+    </dependencies>
+</project>

Review Comment:
   Please add LICENSE / NOTICE files for the NAR bundle.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")
+            .description("Shopify resource type")
+            .required(true)
+            .allowableValues(ResourceDirectory.getCategories())
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getDisplayName())
+                            .description(resourceType.getDescription())
+                            .required(true)
+                            .dependsOn(RESOURCE_TYPE, resourceType.getValue())
+                            .allowableValues(ResourceDirectory.getResourcesAsAllowableValues(resourceType))
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                WEB_CLIENT_PROVIDER,
+                API_URL,
+                ACCESS_TOKEN,
+                API_VERSION,
+                RESOURCE_TYPE
+        ));
+        propertyDescriptors.addAll(resourceDescriptors);
+        return Collections.unmodifiableList(propertyDescriptors);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+    private static final int TOO_MANY_REQUESTS = 429;
+
+    private volatile ShopifyRestService shopifyRestService;
+    private volatile ShopifyResource shopifyResource;
+    private volatile String resourceName;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final WebClientServiceProvider webClientServiceProvider =
+                context.getProperty(WEB_CLIENT_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        final WebClientService webClientService = webClientServiceProvider.getWebClientService();
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder();
+
+        final String apiVersion = context.getProperty(API_VERSION).getValue();
+        final String baseUrl = context.getProperty(API_URL).getValue();
+        final String accessToken = context.getProperty(ACCESS_TOKEN).getValue();
+
+        final String category = context.getProperty(RESOURCE_TYPE).getValue();
+        final ResourceType resourceType = ResourceType.valueOf(category);
+        resourceName = context.getProperty(propertyMap.get(resourceType)).getValue();
+
+        shopifyResource = ResourceDirectory.getResourceTypeDto(resourceType, resourceName);
+
+        shopifyRestService =
+                getShopifyRestService(webClientService, uriBuilder, apiVersion, baseUrl, accessToken, resourceName,
+                        shopifyResource.getIncrementalLoadingParameter());
+    }
+
+    ShopifyRestService getShopifyRestService(final WebClientService webClientService, final HttpUriBuilder uriBuilder,
+            final String apiVersion, final String baseUrl, final String accessToken, final String resourceName,
+            final IncrementalLoadingParameter incrementalLoadingParameter) {
+        return new ShopifyRestService(
+                webClientService,
+                uriBuilder,
+                apiVersion,
+                baseUrl,
+                accessToken,
+                resourceName,
+                incrementalLoadingParameter
+        );
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        return relationships;

Review Comment:
   Please initialize the set once (static field and initializer) instead of creating it every time.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/pom.xml:
##########
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-shopify-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-shopify-processors</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <!-- Test dependencies -->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-service</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-proxy-configuration-api</artifactId>
+            <scope>test</scope>
+        </dependency>

Review Comment:
   This dependency is not used and can be removed.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")

Review Comment:
   The same Display Name and Description should be used as in case of `GetHubSpot`.
   ```suggestion
               .displayName("Access Token")
               .description("Access Token to authenticate requests")
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")

Review Comment:
   `Resource` sounds to me a bit technical and related to the REST API. I would consider to use `Object Category` for this property and `Object Type` for the dependant properties.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")

Review Comment:
   ```suggestion
               .description("The Shopify REST API version")
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)

Review Comment:
   In my understanding it should be required.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")

Review Comment:
   Please remove "NiFi" prefix.
   
   The description could be the same as in case of `GetHubSpot` processor.
   
   ```suggestion
               .displayName("Web Client Service Provider")
               .description("Controller service for HTTP client operations")
   ```



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
+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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceDirectory;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@TriggerWhenEmpty
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters"
+                +
+                "can be found in additional details. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary Node is"
+                +
+                " selected, the new node can pick up where the previous node left off, without duplicating the data.")
+@CapabilityDescription("Retrieves object from a custom Shopify store. The processor yield time must be set to the account's rate limit accordingly.")
+public class GetShopify extends AbstractProcessor {
+
+    public static final PropertyDescriptor WEB_CLIENT_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("NiFi Web Client Service Provider")
+            .description("NiFi Web Client Service Provider to make HTTP calls and build URIs")
+            .required(false)
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .build();
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The API URL of the Custom Shopify App")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN = new PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Admin API Access Token")
+            .description("The Admin API Access Token of the Custom Shopify App")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor API_VERSION = new PropertyDescriptor.Builder()
+            .name("api-version")
+            .displayName("API Version")
+            .description("The used REST API version")
+            .required(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .defaultValue("2022-10")
+            .build();
+
+    static final PropertyDescriptor RESOURCE_TYPE = new PropertyDescriptor.Builder()
+            .name("resource-type")
+            .displayName("Resource Type")
+            .description("Shopify resource type")
+            .required(true)
+            .allowableValues(ResourceDirectory.getCategories())
+            .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("For FlowFiles created as a result of a successful query.")
+            .build();
+
+    private static final Map<ResourceType, PropertyDescriptor> propertyMap = new EnumMap<>(ResourceType.class);
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = createPropertyDescriptors();
+
+    private static List<PropertyDescriptor> createPropertyDescriptors() {
+        final List<PropertyDescriptor> resourceDescriptors = Arrays.stream(ResourceType.values())
+                .map(resourceType -> {
+                    final PropertyDescriptor resourceDescriptor = new PropertyDescriptor.Builder()
+                            .name(resourceType.getValue())
+                            .displayName(resourceType.getDisplayName())
+                            .description(resourceType.getDescription())
+                            .required(true)
+                            .dependsOn(RESOURCE_TYPE, resourceType.getValue())
+                            .allowableValues(ResourceDirectory.getResourcesAsAllowableValues(resourceType))
+                            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+                            .build();
+                    propertyMap.put(resourceType, resourceDescriptor);
+                    return resourceDescriptor;
+                })
+                .collect(Collectors.toList());
+        final List<PropertyDescriptor> propertyDescriptors = new ArrayList<>(Arrays.asList(
+                WEB_CLIENT_PROVIDER,
+                API_URL,
+                ACCESS_TOKEN,
+                API_VERSION,
+                RESOURCE_TYPE
+        ));
+        propertyDescriptors.addAll(resourceDescriptors);
+        return Collections.unmodifiableList(propertyDescriptors);
+    }
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+    private static final int TOO_MANY_REQUESTS = 429;
+
+    private volatile ShopifyRestService shopifyRestService;
+    private volatile ShopifyResource shopifyResource;
+    private volatile String resourceName;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final WebClientServiceProvider webClientServiceProvider =
+                context.getProperty(WEB_CLIENT_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        final WebClientService webClientService = webClientServiceProvider.getWebClientService();
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder();
+
+        final String apiVersion = context.getProperty(API_VERSION).getValue();
+        final String baseUrl = context.getProperty(API_URL).getValue();
+        final String accessToken = context.getProperty(ACCESS_TOKEN).getValue();
+
+        final String category = context.getProperty(RESOURCE_TYPE).getValue();
+        final ResourceType resourceType = ResourceType.valueOf(category);
+        resourceName = context.getProperty(propertyMap.get(resourceType)).getValue();
+
+        shopifyResource = ResourceDirectory.getResourceTypeDto(resourceType, resourceName);
+
+        shopifyRestService =
+                getShopifyRestService(webClientService, uriBuilder, apiVersion, baseUrl, accessToken, resourceName,
+                        shopifyResource.getIncrementalLoadingParameter());
+    }
+
+    ShopifyRestService getShopifyRestService(final WebClientService webClientService, final HttpUriBuilder uriBuilder,
+            final String apiVersion, final String baseUrl, final String accessToken, final String resourceName,
+            final IncrementalLoadingParameter incrementalLoadingParameter) {
+        return new ShopifyRestService(
+                webClientService,
+                uriBuilder,
+                apiVersion,
+                baseUrl,
+                accessToken,
+                resourceName,
+                incrementalLoadingParameter
+        );
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final StateMap state = getState(context);
+        final String fromDateTime = state.get(resourceName);
+
+        final HttpResponseEntity response = shopifyRestService.getShopifyObjects(fromDateTime);
+        final AtomicInteger objectCountHolder = new AtomicInteger();
+
+        if (response.statusCode() == HttpResponseStatus.OK.getCode()) {
+            FlowFile flowFile = session.create();
+            flowFile = session.write(flowFile, parseHttpResponse(response, objectCountHolder));
+            if (objectCountHolder.get() > 0) {
+                session.transfer(flowFile, REL_SUCCESS);
+            } else {
+                getLogger().debug("Empty response when requested Shopify resource: [{}]", resourceName);
+                session.remove(flowFile);
+            }
+        } else if (response.statusCode() >= 400) {
+            if (response.statusCode() == TOO_MANY_REQUESTS) {
+                context.yield();
+                throw new ProcessException(String.format(
+                        "Rate limit exceeded, yielding before retrying request. HTTP %d error for requested URI [%s]",
+                        response.statusCode(), resourceName));
+            } else {
+                context.yield();
+                getLogger().warn("HTTP {} error for requested Shopify resource [{}]", response.statusCode(),
+                        resourceName);
+            }
+        }
+
+        Map<String, String> newState = new HashMap<>(state.toMap());
+        if (shopifyResource.getIncrementalLoadingParameter() != IncrementalLoadingParameter.NONE) {
+            newState.put(shopifyRestService.getResourceName(), getCurrentExecutionTime());

Review Comment:
   I'm afraid it is not enough to get the current time at the end of the processing and store it because new items may be created after the REST call but before this point.
   I think we need to get the current time at the beginning and use it as the upper limit (`created_at_max`) when retrieving objects.



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

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

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


[GitHub] [nifi] Lehel44 commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class ResourceDirectory {
+
+    private static final boolean PAGINATION_SUPPORTED = true;
+    private static final boolean PAGINATION_NOT_SUPPORTED = false;
+
+    public static final List<ShopifyResource> CUSTOMER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "customers",
+                    "Customers",
+                    "The Customer resource stores information about a shop's customers, such as their contact details,"
+                            + " their order history, and whether they've agreed to receive email marketing.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "customer_saved_searches",
+                    "Customer Saved Searches",
+                    "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> DISCOUNT_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "price_rules",
+                    "Price Rules",
+                    "The PriceRule resource can be used to get discounts using conditions",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> INVENTORY_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "locations",
+                    "Locations",
+                    "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> ONLINE_STORE_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "blogs",
+                    "Blogs",
+                    "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "comments",
+                    "Comments",
+                    "A comment is a reader's response to an article in a blog.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "pages",
+                    "Pages",
+                    "Shopify stores come with a tool for creating basic HTML web pages.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "redirects",
+                    "Redirects",
+                    "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "script_tags",
+                    "Script Tags",
+                    "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "themes",
+                    "Themes",
+                    "A theme controls the look and feel of a Shopify online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            )));
+    public static final List<ShopifyResource> ORDER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "checkouts",
+                    "Abandoned Checkouts",
+                    "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "draft_orders",
+                    "Draft Orders",
+                    "Merchants can use draft orders to create orders on behalf of their customers.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "orders",
+                    "Orders",
+                    "An order is a customer's request to purchase one or more products from a shop.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> PRODUCT_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collects",
+                    "Collects",
+                    "Collects are meant for managing the relationship between products and custom collections.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "custom_collections",
+                    "Custom Collections",
+                    "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "products",
+                    "Products",
+                    "Get products in a merchant's store ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "smart_collections",
+                    "Smart Collections",
+                    "A smart collection is a grouping of products defined by rules that are set by the merchant.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collection_listings",
+                    "Collection Listings",
+                    "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "product_listings",
+                    "Product Listings",
+                    "A ProductListing resource represents a Product which is available to your sales channel.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "countries",
+                    "Countries",
+                    "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "currencies",
+                    "Currencies",
+                    "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "policies",
+                    "Policies",
+                    "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "shipping_zones",
+                    "Shipping Zones",
+                    "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(

Review Comment:
   Interesting, directly calling the .../admin/api/2022-10/shipping_zones.json endpoint returns a json. 



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

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

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


[GitHub] [nifi] tpalfy commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/resources/docs/org.apache.nifi.shopify.processors.GetShopify/additionalDetails.html:
##########
@@ -0,0 +1,148 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>GetShopify</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<h2>Setting Up a Custom App</h2>
+<p>
+    Follow the <a href="https://help.shopify.com/en/manual/apps/custom-apps">Shopify tutorial</a> to enable and create
+    private apps, set API Scopes and generate API tokens.
+</p>
+<h2>Incremental Loading</h2>
+<p>
+    Some resources can be processed incrementally by NiFi. This means that only resources created or modified after the
+    last run time of the processor are displayed. The processor state can be reset in the context menu. The following
+    list shows which date-time fields are incremented for which resources.
+<ul>
+    <li>Access
+        <ul>
+            <li>Access Scope: none</li>
+            <li>StoreFront Access Token: none</li>
+        </ul>
+    </li>
+    <li>Analytics
+        <ul>
+            <li>Reports: updated_at_min</li>
+        </ul>
+    </li>
+    <li>Billing
+        <ul>
+            <li>Application Charge: none</li>
+            <li>Application Credit: none</li>
+            <li>Recurring Application Charge: none</li>
+        </ul>
+    </li>
+    <li>Customers
+        <ul>
+            <li>Customers: updated_at_min</li>
+            <li>Customer Saved Searches: none</li>
+        </ul>
+    </li>
+    <li>Discounts
+        <ul>
+            <li>Price Rules: updated_at_min</li>
+        </ul>
+    </li>
+    <li>Events
+        <ul>
+            <li>Events: created_at_min</li>
+        </ul>
+    </li>
+    <li>Inventory
+        <ul>
+            <li>Inventory Levels: updated_at_min</li>
+            <li>Locations: none</li>
+        </ul>
+    </li>
+    <li>Marketing Event
+        <ul>
+            <li>Marketing Events: none</li>
+        </ul>
+    </li>
+    <li>Metafields
+        <ul>
+            <li>Metafields: updated_at_min</li>
+        </ul>
+    </li>
+    <li>Online Store
+        <ul>
+            <li>Blogs: none</li>
+            <li>Comment: none</li>
+            <li>Pages: none</li>
+            <li>Redirects: none</li>
+            <li>Script Tags: updated_at_min</li>
+            <li>Themes: none</li>
+        </ul>
+    </li>
+    <li>Orders
+        <ul>
+            <li>Abandoned Checkouts: updated_at_min</li>
+            <li>Draft Orders: updated_at_min</li>
+            <li>Orders: updated_at_min</li>
+        </ul>
+    </li>
+    <li>Plus
+        <ul>
+            <li>Gift Cards: none</li>
+            <li>Users: none</li>
+        </ul>
+    </li>
+    <li>Product
+        <ul>
+            <li>Collects: none</li>
+            <li>Custom Collections: updated_at_min</li>
+            <li>Products: updated_at_min</li>
+            <li>Smart Collections: updated_at_min</li>
+        </ul>
+    </li>
+    <li>Sales Channels
+        <ul>
+            <li>Collection Listings: none</li>
+            <li>Mobile Platform Applications: none</li>
+            <li>Product Listings: updated_at_min</li>
+            <li>Resource Feedbacks: none</li>
+        </ul>
+    </li>
+    <li>Shipping and Fulfillments
+        <ul>
+            <li>Carrier Services: none</li>
+        </ul>
+    </li>
+    <li>Store Properties
+        <ul>
+            <li>Countries: none</li>
+            <li>Currencies: none</li>
+            <li>Policies: none</li>
+            <li>Shipping Zones: updated_at_min</li>
+            <li>Shop: none</li>
+        </ul>
+    </li>
+    <li>Tender Transactions
+        <ul>
+            <li>Tender Transactions: processed_at_min</li>
+        </ul>
+    </li>
+</ul>
+
+Specific trap type can set in case of Enterprise Specific generic trap type is chosen.

Review Comment:
   I don't think we need trap-related information here.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/resources/docs/org.apache.nifi.shopify.processors.GetShopify/additionalDetails.html:
##########
@@ -0,0 +1,148 @@
+<!DOCTYPE html>

Review Comment:
   Should be under `org.apache.nifi.processors.shopify.GetShopify` (`shopify` is missing).



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class ResourceDirectory {
+
+    private static final boolean PAGINATION_SUPPORTED = true;
+    private static final boolean PAGINATION_NOT_SUPPORTED = false;
+
+    public static final List<ShopifyResource> CUSTOMER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "customers",
+                    "Customers",
+                    "The Customer resource stores information about a shop's customers, such as their contact details,"
+                            + " their order history, and whether they've agreed to receive email marketing.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "customer_saved_searches",
+                    "Customer Saved Searches",
+                    "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> DISCOUNT_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "price_rules",
+                    "Price Rules",
+                    "The PriceRule resource can be used to get discounts using conditions",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> INVENTORY_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "locations",
+                    "Locations",
+                    "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> ONLINE_STORE_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "blogs",
+                    "Blogs",
+                    "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "comments",
+                    "Comments",
+                    "A comment is a reader's response to an article in a blog.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "pages",
+                    "Pages",
+                    "Shopify stores come with a tool for creating basic HTML web pages.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "redirects",
+                    "Redirects",
+                    "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "script_tags",
+                    "Script Tags",
+                    "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "themes",
+                    "Themes",
+                    "A theme controls the look and feel of a Shopify online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            )));
+    public static final List<ShopifyResource> ORDER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "checkouts",
+                    "Abandoned Checkouts",
+                    "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "draft_orders",
+                    "Draft Orders",
+                    "Merchants can use draft orders to create orders on behalf of their customers.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "orders",
+                    "Orders",
+                    "An order is a customer's request to purchase one or more products from a shop.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> PRODUCT_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collects",
+                    "Collects",
+                    "Collects are meant for managing the relationship between products and custom collections.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "custom_collections",
+                    "Custom Collections",
+                    "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "products",
+                    "Products",
+                    "Get products in a merchant's store ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "smart_collections",
+                    "Smart Collections",
+                    "A smart collection is a grouping of products defined by rules that are set by the merchant.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collection_listings",
+                    "Collection Listings",
+                    "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "product_listings",
+                    "Product Listings",
+                    "A ProductListing resource represents a Product which is available to your sales channel.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "countries",
+                    "Countries",
+                    "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "currencies",
+                    "Currencies",
+                    "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "policies",
+                    "Policies",
+                    "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "shipping_zones",
+                    "Shipping Zones",
+                    "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(

Review Comment:
   When I tested, `Shop` returned a zip and the current implementation can only handle JSON responses.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/IncrementalLoadingParameter.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.shopify.model;
+
+public enum IncrementalLoadingParameter {
+    CREATED_AT,
+    UPDATED_AT,
+    PUBLISHED_AT,
+    PROCESSED_AT,

Review Comment:
   ```suggestion
   ```
   They are not used.



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/resources/docs/org.apache.nifi.shopify.processors.GetShopify/additionalDetails.html:
##########
@@ -0,0 +1,148 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>GetShopify</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+
+<body>
+<h2>Setting Up a Custom App</h2>
+<p>
+    Follow the <a href="https://help.shopify.com/en/manual/apps/custom-apps">Shopify tutorial</a> to enable and create
+    private apps, set API Scopes and generate API tokens.
+</p>
+<h2>Incremental Loading</h2>
+<p>
+    Some resources can be processed incrementally by NiFi. This means that only resources created or modified after the
+    last run time of the processor are displayed. The processor state can be reset in the context menu. The following
+    list shows which date-time fields are incremented for which resources.
+<ul>
+    <li>Access
+        <ul>
+            <li>Access Scope: none</li>

Review Comment:
   Wouldn't it be better to just leave out the ones that don't support incremental loading?



##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/GetShopify.java:
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.shopify;
+
+import com.fasterxml.jackson.core.JsonEncoding;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.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.state.Scope;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.shopify.model.IncrementalLoadingParameter;
+import org.apache.nifi.processors.shopify.model.ResourceType;
+import org.apache.nifi.processors.shopify.model.ShopifyResource;
+import org.apache.nifi.processors.shopify.rest.ShopifyRestService;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpResponseStatus;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@Tags({"shopify"})
+@Stateful(scopes = Scope.CLUSTER, description =
+        "For a few resources the processors support incremental loading. The list of the resources with the supported parameters" +
+                " can be found in additional details.")

Review Comment:
   ```suggestion
           "For a few resources the processor supports incremental loading. The list of the resources with the supported parameters" +
                   " can be found in the additional details.")
   ```



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

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

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


[GitHub] [nifi] Lehel44 commented on a diff in pull request #6303: NIFI-10213: Create GetShopify processor

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


##########
nifi-nar-bundles/nifi-shopify-bundle/nifi-shopify-processors/src/main/java/org/apache/nifi/processors/shopify/model/ResourceDirectory.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.shopify.model;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class ResourceDirectory {
+
+    private static final boolean PAGINATION_SUPPORTED = true;
+    private static final boolean PAGINATION_NOT_SUPPORTED = false;
+
+    public static final List<ShopifyResource> CUSTOMER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "customers",
+                    "Customers",
+                    "The Customer resource stores information about a shop's customers, such as their contact details,"
+                            + " their order history, and whether they've agreed to receive email marketing.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "customer_saved_searches",
+                    "Customer Saved Searches",
+                    "A customer saved search is a search query that represents a group of customers defined by the shop owner.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> DISCOUNT_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "price_rules",
+                    "Price Rules",
+                    "The PriceRule resource can be used to get discounts using conditions",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> INVENTORY_RESOURCES =
+            Collections.singletonList(ShopifyResource.newInstance(
+                    "locations",
+                    "Locations",
+                    "A location represents a geographical location where your stores, pop-up stores, headquarters and warehouses exist.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ));
+    public static final List<ShopifyResource> ONLINE_STORE_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "blogs",
+                    "Blogs",
+                    "Shopify shops come with a built-in blogging engine, allowing a shop to have one or more blogs.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "comments",
+                    "Comments",
+                    "A comment is a reader's response to an article in a blog.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "pages",
+                    "Pages",
+                    "Shopify stores come with a tool for creating basic HTML web pages.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "redirects",
+                    "Redirects",
+                    "A redirect causes a visitor on a specific path on the shop's site to be automatically sent to a different location.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "script_tags",
+                    "Script Tags",
+                    "The ScriptTag resource represents remote JavaScript code that is loaded into the pages of a shop's storefront or the order status page of checkout.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "themes",
+                    "Themes",
+                    "A theme controls the look and feel of a Shopify online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            )));
+    public static final List<ShopifyResource> ORDER_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "checkouts",
+                    "Abandoned Checkouts",
+                    "A checkout is considered abandoned after the customer has added contact information, but before the customer has completed their purchase.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "draft_orders",
+                    "Draft Orders",
+                    "Merchants can use draft orders to create orders on behalf of their customers.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "orders",
+                    "Orders",
+                    "An order is a customer's request to purchase one or more products from a shop.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> PRODUCT_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collects",
+                    "Collects",
+                    "Collects are meant for managing the relationship between products and custom collections.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "custom_collections",
+                    "Custom Collections",
+                    "A custom collection is a grouping of products that a merchant can create to make their store easier to browse. ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "products",
+                    "Products",
+                    "Get products in a merchant's store ",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "smart_collections",
+                    "Smart Collections",
+                    "A smart collection is a grouping of products defined by rules that are set by the merchant.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> SALES_CHANNEL_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "collection_listings",
+                    "Collection Listings",
+                    "A CollectionListing resource represents a product collection that a merchant has made available to your sales channel.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "product_listings",
+                    "Product Listings",
+                    "A ProductListing resource represents a Product which is available to your sales channel.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_SUPPORTED
+            )));
+    public static final List<ShopifyResource> STORE_PROPERTY_RESOURCES =
+            Collections.unmodifiableList(Arrays.asList(ShopifyResource.newInstance(
+                    "countries",
+                    "Countries",
+                    "The Country resource represents the tax rates applied to orders from the different countries where a shop sells its products.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "currencies",
+                    "Currencies",
+                    "Merchants who use Shopify Payments can allow customers to pay in their local currency on the online store.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "policies",
+                    "Policies",
+                    "Policy resource can be used to access the policies that a merchant has configured for their shop, such as their refund and privacy policies.",
+                    IncrementalLoadingParameter.NONE,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(
+                    "shipping_zones",
+                    "Shipping Zones",
+                    "ShippingZone resource can be used to view shipping zones and their countries, provinces, and shipping rates.",
+                    IncrementalLoadingParameter.UPDATED_AT,
+                    PAGINATION_NOT_SUPPORTED
+            ), ShopifyResource.newInstance(

Review Comment:
   Interesting, directly calling the .../admin/api/2022-10/shipping_zones.json endpoint returns a json. 



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