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/30 10:26:29 UTC

[GitHub] [nifi] dam4rus opened a new pull request, #6350: NIFI-10242 Create QueryAirtableTable processor

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

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


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

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

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


[GitHub] [nifi] dam4rus commented on a diff in pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/JsonRecordReaderFactory.java:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.serialization;
+
+public interface JsonRecordReaderFactory extends RecordReaderFactory {
+
+}

Review Comment:
   It's related to the comment left by @turcsanyip https://github.com/apache/nifi/pull/6350#pullrequestreview-1091713871
   
   In it's current implementation we need a `JsonTreeReader` CS to read the schema of the records. Since Airtable returns a JSON response it made sense to limit the available choices for the "Schema Reader" to only `JsonTreeReader` and `PropertyDescriptor.identifiesControllerService` seems to work only with interfaces. That's why I created a new interface.
   
   If we move away from using a `JsonTreeReader` to parse the schema this interface can be removed



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/JsonRecordReaderFactory.java:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.serialization;
+
+public interface JsonRecordReaderFactory extends RecordReaderFactory {
+
+}

Review Comment:
   @dam4rus This interface can now be deleted because the Recored Reader approach has been reverted.



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per Flow File")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("output-batch-size")
+            .displayName("Output Batch Size")
+            .description("The number of output FlowFiles to queue before committing the process session. When set to zero, the session will be committed when all records"
+                    + " have been processed and the output FlowFiles are ready for transfer to the downstream relationship. For large result sets, this can cause a large burst of FlowFiles"
+                    + " to be transferred at the end of processor execution. If this property is set, then when the specified number of FlowFiles are ready for transfer, then the session will"
+                    + " be committed, thus releasing the FlowFiles to the downstream relationship.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
+            .name("metadata-strategy")
+            .displayName("Metadata Strategy")
+            .description("Strategy to use for fetching record schema. Currently only 'Use JSON Record Reader' is supported."
+                    + " When Airtable Metadata API becomes more stable it will be possible to fetch the record schema through it.")
+            .required(true)
+            .defaultValue(MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .allowableValues(MetadataStrategy.class)
+            .build();
+
+    static final PropertyDescriptor SCHEMA_READER = new PropertyDescriptor.Builder()
+            .name("schema-reader")
+            .displayName("Schema Reader")
+            .description("JsonTreeReader service to use for fetching the schema of records returned by the Airtable REST API")
+            .dependsOn(METADATA_STRATEGY, MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .identifiesControllerService(JsonRecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Service used for writing records returned by the Airtable REST API")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            OUTPUT_BATCH_SIZE,
+            METADATA_STRATEGY,
+            SCHEMA_READER,
+            RECORD_WRITER,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider.getWebClientService(), apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final JsonRecordReaderFactory schemaRecordReaderFactory = context.getProperty(SCHEMA_READER).asControllerService(JsonRecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+        final Integer outputBatchSize = context.getProperty(OUTPUT_BATCH_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now().minusSeconds(QUERY_LAG_SECONDS).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final byte[] recordsJson;
+        try {
+            recordsJson = airtableRestService.getRecords(getRecordsParameters);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("REST API rate limit exceeded while fetching initial Airtable record set", e);
+        }
+
+        final FlowFile flowFile = session.create();
+        final Map<String, String> originalAttributes = flowFile.getAttributes();
+
+        final RecordSchema recordSchema;
+        final RecordSchema writerSchema;
+        try {
+            final ByteArrayInputStream recordsStream = new ByteArrayInputStream(recordsJson);
+            recordSchema = schemaRecordReaderFactory.createRecordReader(flowFile, recordsStream, getLogger()).getSchema();
+            writerSchema = writerFactory.getSchema(originalAttributes, recordSchema);
+        } catch (MalformedRecordException | IOException | SchemaNotFoundException e) {
+            throw new ProcessException("Couldn't get record schema", e);
+        }
+
+        session.remove(flowFile);
+
+        final List<FlowFile> flowFiles = new ArrayList<>();
+        int totalRecordCount = 0;
+        final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory = new AirtableJsonTreeRowRecordReaderFactory(getLogger(), recordSchema);
+        try (final AirtableRecordSet airtableRecordSet = new AirtableRecordSet(recordsJson, recordReaderFactory, airtableRestService, getRecordsParameters)) {
+            while (true) {
+                final AtomicInteger recordCountHolder = new AtomicInteger();
+                final Map<String, String> attributes = new HashMap<>();
+                FlowFile flowFileToAdd = session.create();
+                flowFileToAdd = session.write(flowFileToAdd, out -> {
+                    try (final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writerSchema, out, originalAttributes)) {
+                        final RecordSet recordSet = (maxRecordsPerFlowFile > 0 ? airtableRecordSet.limit(maxRecordsPerFlowFile) : airtableRecordSet);
+                        final WriteResult writeResult = writer.write(recordSet);
+
+                        attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+                        attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
+                        attributes.putAll(writeResult.getAttributes());
+
+                        recordCountHolder.set(writeResult.getRecordCount());
+                    } catch (final IOException e) {
+                        final Throwable cause = e.getCause();
+                        if (cause instanceof RateLimitExceededException) {
+                            context.yield();
+                            throw new ProcessException("REST API rate limit exceeded while reading Airtable records", cause);
+                        }
+                        throw new ProcessException("Couldn't read records from input", e);
+                    } catch (final SchemaNotFoundException e) {
+                        throw new ProcessException("Couldn't read records from input", e);
+                    }
+                });
+
+                flowFileToAdd = session.putAllAttributes(flowFileToAdd, attributes);
+
+                final int recordCount = recordCountHolder.get();
+                if (recordCount > 0) {
+                    totalRecordCount += recordCount;
+                    flowFiles.add(flowFileToAdd);
+                    if (outputBatchSize > 0 && flowFiles.size() == outputBatchSize) {
+                        transferFlowFiles(flowFiles, session, totalRecordCount);
+                        flowFiles.clear();
+                    }
+                    continue;
+                }
+
+                session.remove(flowFileToAdd);
+
+                if (maxRecordsPerFlowFile > 0 && outputBatchSize == 0) {
+                    fragmentFlowFiles(session, flowFiles);
+                }
+
+                transferFlowFiles(flowFiles, session, totalRecordCount);
+                break;
+            }
+        } catch (final IOException e) {
+            throw new ProcessException("Couldn't read records from input", e);
+        }
+
+        if (totalRecordCount == 0) {
+            return;
+        }
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to update cluster state", e);
+        }
+    }
+
+    private AirtableGetRecordsParameters buildGetRecordsParameters(final ProcessContext context,
+            final String lastRecordFetchTime,
+            final String nowDateTimeString) {
+        final String fieldsProperty = context.getProperty(FIELDS).evaluateAttributeExpressions().getValue();
+        final String customFilter = context.getProperty(CUSTOM_FILTER).evaluateAttributeExpressions().getValue();
+        final Integer pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final AirtableGetRecordsParameters.Builder getRecordsParametersBuilder = new AirtableGetRecordsParameters.Builder();
+        if (lastRecordFetchTime != null) {
+            getRecordsParametersBuilder
+                    .modifiedAfter(lastRecordFetchTime)
+                    .modifiedBefore(nowDateTimeString);
+        }
+        if (fieldsProperty != null) {
+            getRecordsParametersBuilder.fields(Arrays.stream(fieldsProperty.split(",")).map(String::trim).collect(Collectors.toList()));
+        }
+        getRecordsParametersBuilder.customFilter(customFilter);
+        if (pageSize != null && pageSize > 0) {
+            getRecordsParametersBuilder.pageSize(pageSize);
+        }
+
+        return getRecordsParametersBuilder.build();
+    }
+
+    private void fragmentFlowFiles(final ProcessSession session, final List<FlowFile> flowFiles) {
+        final String fragmentIdentifier = UUID.randomUUID().toString();
+        for (int i = 0; i < flowFiles.size(); i++) {
+            final Map<String, String> fragmentAttributes = new HashMap<>();
+            fragmentAttributes.put(FRAGMENT_ID.key(), fragmentIdentifier);

Review Comment:
   Please mention the "fragment.*" attributes as well in @WritesAttributes.



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per Flow File' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet. If Output Batch Size is set, then this "
+                + "attribute will not be populated."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per Flow File' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order  "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(QUERY_LAG_SECONDS)

Review Comment:
   This 1 second query lag handles the second-level time precision of the records (new records may be created after the query execution but still within the same second) but it is also possible to miss records due to the unsynchronized server / client system times.
   If NiFi's time is ahead of Airtable's time and NiFi sends its current system time as the upper limit of the query, then Airtable can create records with timestamps falling into this time window (lower timestamp than this upper limit) after the query was executed. NiFi will use this timestamp as the lower limit for the time wondow in the next round, so these new records will never be selected and be missed.
   I'm afraid there is no way to retrieve Airtable's time, so we need to make this lag configurable and the user can set what time lag they want to handle (similar to the Min Object Age of the List* processors). The price is the latency because the longer the time lag, the higher the latency (new records have to wait for being old enough).
   Suggested property name: `Query Time Window Lag`
   Suggested default value: `3 s`



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per Flow File' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet. If Output Batch Size is set, then this "
+                + "attribute will not be populated."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per Flow File' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order  "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(QUERY_LAG_SECONDS)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final AirtableRetrieveTableResult retrieveTableResult;
+        try {
+            final AirtableTableRetriever tableRetriever = new AirtableTableRetriever(airtableRestService, getRecordsParameters, maxRecordsPerFlowFile);
+            retrieveTableResult = tableRetriever.retrieveAll(session);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to read Airtable records", e);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("Airtable REST API rate limit exceeded while reading records", e);
+        }
+
+        final List<FlowFile> flowFiles = retrieveTableResult.getFlowFiles();
+        if (flowFiles.isEmpty()) {
+            return;
+        }
+
+        if (maxRecordsPerFlowFile != null && maxRecordsPerFlowFile > 0) {
+            fragmentFlowFiles(session, flowFiles);
+        }
+        transferFlowFiles(session, flowFiles, retrieveTableResult.getTotalRecordCount());
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to update cluster state", e);
+        }
+    }
+
+    private AirtableGetRecordsParameters buildGetRecordsParameters(final ProcessContext context,
+            final String lastRecordFetchTime,
+            final String nowDateTimeString) {
+        final String fieldsProperty = context.getProperty(FIELDS).evaluateAttributeExpressions().getValue();
+        final String customFilter = context.getProperty(CUSTOM_FILTER).evaluateAttributeExpressions().getValue();
+        final Integer pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final AirtableGetRecordsParameters.Builder getRecordsParametersBuilder = new AirtableGetRecordsParameters.Builder();
+        if (lastRecordFetchTime != null) {
+            getRecordsParametersBuilder
+                    .modifiedAfter(lastRecordFetchTime)
+                    .modifiedBefore(nowDateTimeString);

Review Comment:
   We need to set `modifiedBefore` even at the first execution (when no `lastRecordFetchTime` is present).



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)

Review Comment:
   Value `0` for configuring the default page size seems to me a bit technical approach (though I know other processors use it).
   I'd suggest making this property optional with valid values 1..100 and using "no value" for the default page size. 



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)

Review Comment:
   Similar to Page Size, it should be an optional property and `0` should not be accepted as a valid value.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            QUERY_TIME_WINDOW_LAG,

Review Comment:
   I'd suggest to move `Page Size` after `Query Time Window Lag` because Lag is more important and mandatory.
   
   It could also be renamed to `Query Page Size` to make it clear what it is used for.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/resources/docs/org.apache.nifi.processors.airtable.QueryAirtableTable/additionalDetails.html:
##########
@@ -0,0 +1,61 @@
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE html>
+<html lang="en" xmlns="http://www.w3.org/1999/html">
+<head>
+    <meta charset="utf-8"/>
+    <title>QueryAirtableTable</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+    <style>
+h2 {margin-top: 4em}
+h3 {margin-top: 3em}
+td {text-align: left}
+    </style>
+</head>
+
+<body>
+
+<h1>QueryAirtableTable</h1>
+
+<h3>Description</h3>
+<p>
+    Airtable is a spreadsheet-database hybrid. In Airtable an application is called base and each base can have multiple tables.
+    A table consists of records (rows) and each record can have multiple fields (columns).
+    The QueryAirtableTable processor can query records from a single base and table via Airtable's REST API.
+    The processor utilizes streams to be able to handle a large number of records.
+    It can also split large record sets to multiple FlowFiles just like a database processor.
+</p>
+
+<h3>API Key</h3>
+<p>
+    Airtable REST API calls requires an API Key that needs to be passed in a request. An Airtable account is required to generate an API Key.
+</p>
+
+<h3>API rate limit</h3>
+<p>
+    The Airtable REST API limits the number of requests that can be sent on a per-base basis to avoid bottlenecks. Currently, this limit is 5 requests per second per base.
+    If this limit is exceeded you can't make another request for 30 seconds. It's your responsibility to handle this rate limit.

Review Comment:
   I think we should provide some hints on it, like:
   ```suggestion
       If this limit is exceeded you can't make another request for 30 seconds. It's your responsibility to handle this rate limit via configuring Yield Duration and Run Schedule properly. It is recommended to start off with the default settings and to increase both parameters when rate limit issues occur.
   ```
   



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            QUERY_TIME_WINDOW_LAG,
+            MAX_RECORDS_PER_FLOWFILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOWFILE).evaluateAttributeExpressions().asInteger();
+        final Long queryTimeWindowLagSeconds = context.getProperty(QUERY_TIME_WINDOW_LAG).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS);
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(queryTimeWindowLagSeconds)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final AirtableRetrieveTableResult retrieveTableResult;
+        try {
+            final AirtableTableRetriever tableRetriever = new AirtableTableRetriever(airtableRestService, getRecordsParameters, maxRecordsPerFlowFile);
+            retrieveTableResult = tableRetriever.retrieveAll(session);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to read Airtable records", e);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("Airtable REST API rate limit exceeded while reading records", e);
+        }
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to update cluster state", e);
+        }
+
+        final List<FlowFile> flowFiles = retrieveTableResult.getFlowFiles();
+        if (flowFiles.isEmpty()) {
+            context.yield();
+            return;
+        }
+
+        if (maxRecordsPerFlowFile != null && maxRecordsPerFlowFile > 0) {
+            fragmentFlowFiles(session, flowFiles);
+        }
+        final String transitUri = airtableRestService.createUriBuilder(false).build().toString();
+        transferFlowFiles(session, flowFiles, retrieveTableResult.getTotalRecordCount(), transitUri);
+    }
+
+    private AirtableGetRecordsParameters buildGetRecordsParameters(final ProcessContext context,
+            final String lastRecordFetchTime,
+            final String nowDateTimeString) {
+        Objects.requireNonNull(context);
+        Objects.requireNonNull(nowDateTimeString);
+
+        final String fieldsProperty = context.getProperty(FIELDS).evaluateAttributeExpressions().getValue();
+        final String customFilter = context.getProperty(CUSTOM_FILTER).evaluateAttributeExpressions().getValue();
+        final Integer pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final AirtableGetRecordsParameters.Builder getRecordsParametersBuilder = new AirtableGetRecordsParameters.Builder();
+        if (lastRecordFetchTime != null) {
+            getRecordsParametersBuilder.modifiedAfter(lastRecordFetchTime);
+        }
+        getRecordsParametersBuilder.modifiedBefore(nowDateTimeString);
+        if (fieldsProperty != null) {
+            getRecordsParametersBuilder.fields(Arrays.stream(fieldsProperty.split(",")).map(String::trim).collect(Collectors.toList()));
+        }
+        getRecordsParametersBuilder.customFilter(customFilter);
+        if (pageSize != null && pageSize > 0) {
+            getRecordsParametersBuilder.pageSize(pageSize);
+        }
+
+        return getRecordsParametersBuilder.build();
+    }
+
+    private void fragmentFlowFiles(final ProcessSession session, final List<FlowFile> flowFiles) {

Review Comment:
   The FlowFiles are already fragmented. For this reason, `addFragmentAttributes()` method name would be better.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            QUERY_TIME_WINDOW_LAG,
+            MAX_RECORDS_PER_FLOWFILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";

Review Comment:
   To make the labels on the UI more consistent, it should be called `last_query_time_window_end` (similar to the `Query Time Window Lag` property).
   
   The constant / local variable names can be adjusted accordingly, but the main point here is to make it more understandable for the user.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            QUERY_TIME_WINDOW_LAG,
+            MAX_RECORDS_PER_FLOWFILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOWFILE).evaluateAttributeExpressions().asInteger();
+        final Long queryTimeWindowLagSeconds = context.getProperty(QUERY_TIME_WINDOW_LAG).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS);
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(queryTimeWindowLagSeconds)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);

Review Comment:
   As the Last Modified field of the records is stored in second precision in Airtable, I recommend using the same precision for the query time window as well. It is also more readable for humans and can make troubleshooting a bit easier.
   ```suggestion
           final String currentRecordFetchDateTime = OffsetDateTime.now()
                   .minusSeconds(queryTimeWindowLagSeconds)
                   .truncatedTo(ChronoUnit.SECONDS)
                   .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"

Review Comment:
   ```suggestion
               .description("The amount of lag to be applied to the query time window's end point. Set this property to avoid missing records when the clock of your local machines"
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            QUERY_TIME_WINDOW_LAG,
+            MAX_RECORDS_PER_FLOWFILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOWFILE).evaluateAttributeExpressions().asInteger();
+        final Long queryTimeWindowLagSeconds = context.getProperty(QUERY_TIME_WINDOW_LAG).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS);
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(queryTimeWindowLagSeconds)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final AirtableRetrieveTableResult retrieveTableResult;
+        try {
+            final AirtableTableRetriever tableRetriever = new AirtableTableRetriever(airtableRestService, getRecordsParameters, maxRecordsPerFlowFile);
+            retrieveTableResult = tableRetriever.retrieveAll(session);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to read Airtable records", e);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("Airtable REST API rate limit exceeded while reading records", e);
+        }
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to update cluster state", e);
+        }
+
+        final List<FlowFile> flowFiles = retrieveTableResult.getFlowFiles();
+        if (flowFiles.isEmpty()) {
+            context.yield();
+            return;
+        }
+
+        if (maxRecordsPerFlowFile != null && maxRecordsPerFlowFile > 0) {
+            fragmentFlowFiles(session, flowFiles);
+        }
+        final String transitUri = airtableRestService.createUriBuilder(false).build().toString();

Review Comment:
   I think we can add the port in `transitUri` as well (in most cases it will not be present either). So the boolean flag is not needed.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableRestService.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.airtable.service;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.Range;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+public class AirtableRestService {
+
+    public static final String API_V0_BASE_URL = "https://api.airtable.com/v0";
+
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Range<Integer> SUCCESSFUL_RESPONSE_RANGE = Range.between(200, 299);
+
+    private final WebClientServiceProvider webClientServiceProvider;
+    private final String apiUrl;
+    private final String apiKey;
+    private final String baseId;
+    private final String tableId;
+
+    public AirtableRestService(final WebClientServiceProvider webClientServiceProvider,
+            final String apiUrl,
+            final String apiKey,
+            final String baseId,
+            final String tableId) {
+        this.webClientServiceProvider = webClientServiceProvider;
+        this.apiUrl = apiUrl;
+        this.apiKey = apiKey;
+        this.baseId = baseId;
+        this.tableId = tableId;
+    }
+
+    public <R> R getRecords(final AirtableGetRecordsParameters getRecordsParameters, final Function<InputStream, R> callback) {
+        final URI uri = buildUri(getRecordsParameters);
+        try (final HttpResponseEntity response = webClientServiceProvider.getWebClientService()
+                .get()
+                .uri(uri)
+                .header("Authorization", "Bearer " + apiKey)
+                .retrieve()) {
+
+            final InputStream bodyInputStream = response.body();
+            if (SUCCESSFUL_RESPONSE_RANGE.contains(response.statusCode())) {
+                return callback.apply(bodyInputStream);
+            }
+            if (response.statusCode() == TOO_MANY_REQUESTS) {
+                throw new RateLimitExceededException();
+            }
+            final StringBuilder exceptionMessageBuilder = new StringBuilder("Error response. Code: " + response.statusCode());
+            final String bodyText = IOUtils.toString(bodyInputStream, StandardCharsets.UTF_8);
+            if (bodyText != null) {
+                exceptionMessageBuilder.append(" Body: ").append(bodyText);
+            }
+
+            throw new ProcessException(exceptionMessageBuilder.toString());
+        } catch (IOException e) {
+            throw new RuntimeException(String.format("Airtable HTTP request failed [%s]", uri), e);

Review Comment:
   ```suggestion
               throw new ProcessException(String.format("Airtable HTTP request failed [%s]", uri), e);
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            QUERY_TIME_WINDOW_LAG,
+            MAX_RECORDS_PER_FLOWFILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOWFILE).evaluateAttributeExpressions().asInteger();
+        final Long queryTimeWindowLagSeconds = context.getProperty(QUERY_TIME_WINDOW_LAG).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS);
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(queryTimeWindowLagSeconds)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final AirtableRetrieveTableResult retrieveTableResult;
+        try {
+            final AirtableTableRetriever tableRetriever = new AirtableTableRetriever(airtableRestService, getRecordsParameters, maxRecordsPerFlowFile);
+            retrieveTableResult = tableRetriever.retrieveAll(session);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to read Airtable records", e);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("Airtable REST API rate limit exceeded while reading records", e);
+        }
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);

Review Comment:
   Please call `ProcessSession.setState()` instead. The `StateManager` returned by `context.getStateManager()` is not transactional and the state gets saved immediately after `setState()` is called. When records found, the state must be saved together with the outgoing FlowFiles when the session ends successfully (commit).
   ```suggestion
               session.setState(newState, Scope.CLUSTER);
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to add to the query time window. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            QUERY_TIME_WINDOW_LAG,
+            MAX_RECORDS_PER_FLOWFILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOWFILE).evaluateAttributeExpressions().asInteger();
+        final Long queryTimeWindowLagSeconds = context.getProperty(QUERY_TIME_WINDOW_LAG).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS);
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);

Review Comment:
   ```suggestion
               state = session.getState(Scope.CLUSTER);
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableRestService.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.airtable.service;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.Range;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+public class AirtableRestService {
+
+    public static final String API_V0_BASE_URL = "https://api.airtable.com/v0";
+
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Range<Integer> SUCCESSFUL_RESPONSE_RANGE = Range.between(200, 299);
+
+    private final WebClientServiceProvider webClientServiceProvider;
+    private final String apiUrl;
+    private final String apiKey;
+    private final String baseId;
+    private final String tableId;
+
+    public AirtableRestService(final WebClientServiceProvider webClientServiceProvider,
+            final String apiUrl,
+            final String apiKey,
+            final String baseId,
+            final String tableId) {
+        this.webClientServiceProvider = webClientServiceProvider;
+        this.apiUrl = apiUrl;
+        this.apiKey = apiKey;
+        this.baseId = baseId;
+        this.tableId = tableId;
+    }
+
+    public <R> R getRecords(final AirtableGetRecordsParameters getRecordsParameters, final Function<InputStream, R> callback) {
+        final URI uri = buildUri(getRecordsParameters);
+        try (final HttpResponseEntity response = webClientServiceProvider.getWebClientService()
+                .get()
+                .uri(uri)
+                .header("Authorization", "Bearer " + apiKey)
+                .retrieve()) {
+
+            final InputStream bodyInputStream = response.body();
+            if (SUCCESSFUL_RESPONSE_RANGE.contains(response.statusCode())) {
+                return callback.apply(bodyInputStream);
+            }
+            if (response.statusCode() == TOO_MANY_REQUESTS) {
+                throw new RateLimitExceededException();
+            }
+            final StringBuilder exceptionMessageBuilder = new StringBuilder("Error response. Code: " + response.statusCode());
+            final String bodyText = IOUtils.toString(bodyInputStream, StandardCharsets.UTF_8);
+            if (bodyText != null) {
+                exceptionMessageBuilder.append(" Body: ").append(bodyText);
+            }
+
+            throw new ProcessException(exceptionMessageBuilder.toString());
+        } catch (IOException e) {
+            throw new RuntimeException(String.format("Airtable HTTP request failed [%s]", uri), e);
+        }
+    }
+
+    public HttpUriBuilder createUriBuilder(final boolean includePort) {
+        final URI uri = URI.create(apiUrl);
+        final HttpUriBuilder uriBuilder = webClientServiceProvider.getHttpUriBuilder()
+                .scheme(uri.getScheme())
+                .host(uri.getHost())
+                .encodedPath(uri.getPath())
+                .addPathSegment(baseId)
+                .addPathSegment(tableId);
+        if (includePort && uri.getPort() != -1) {
+            uriBuilder.port(uri.getPort());
+        }
+        return uriBuilder;
+    }
+
+    private URI buildUri(AirtableGetRecordsParameters getRecordsParameters) {
+        final HttpUriBuilder uriBuilder = createUriBuilder(true);
+        for (final String field : getRecordsParameters.getFields()) {
+            uriBuilder.addQueryParameter("fields[]", field);
+        }
+
+        final List<String> filters = new ArrayList<>();
+        getRecordsParameters.getCustomFilter()
+                .ifPresent(filters::add);
+        getRecordsParameters.getModifiedAfter()
+                .map(modifiedAfter -> "IS_AFTER(LAST_MODIFIED_TIME(),DATETIME_PARSE(\"" + modifiedAfter + "\"))")
+                .ifPresent(filters::add);
+        getRecordsParameters.getModifiedBefore()
+                .map(modifiedBefore -> "IS_BEFORE(LAST_MODIFIED_TIME(),DATETIME_PARSE(\"" + modifiedBefore + "\"))")
+                .ifPresent(filters::add);

Review Comment:
   As far as I can see (please double check it), both `IS_AFTER` and `IS_BEFORE` are exclusive but one endpoint of the query time window must be inclusive in order not to loose records with Last Modified time equal to the time window start/end. The start point should be inclusive.



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,330 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoUnit;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " State is stored across the cluster, so this Processor can run only on the Primary Node and if a new Primary Node is selected,"
+        + " the new node can pick up where the previous one left off without duplicating the data.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per FlowFile' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per FlowFile' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per FlowFile' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "15 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor QUERY_TIME_WINDOW_LAG = new PropertyDescriptor.Builder()
+            .name("query-time-window-lag")
+            .displayName("Query Time Window Lag")
+            .description("The amount of lag to be applied to the query time window's end point. Set this property to avoid missing records when the clock of your local machines"
+                    + " and Airtable servers' clock are not in sync. Must be greater than or equal to 1 second.")
+            .defaultValue("3 s")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor QUERY_PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("query-page-size")
+            .displayName("Query Page Size")
+            .description("Number of records to be fetched in a page. Should be between 1 and 100 inclusively.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(1, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOWFILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flowfile")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")

Review Comment:
   Minor documentation fix (it can be fixed later or during the merge if no new commit is needed in this PR)
    
   ```suggestion
                       + " result sets into multiple FlowFiles. If no value specified, then all records are returned in a single FlowFile.")
   ```



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

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

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


[GitHub] [nifi] turcsanyip commented on pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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

   @exceptionfactory @krisztina-zsihovszki Thanks for the reviews!
   
   Merging 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] exceptionfactory commented on a diff in pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/record/AirtableRecordSet.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.airtable.record;
+
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.Option;
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import org.apache.nifi.json.JsonTreeRowRecordReader;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceeded;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+public class AirtableRecordSet implements RecordSet, Closeable {
+
+    final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory;
+    final AirtableRestService restService;
+    final AirtableGetRecordsParameters getRecordsParameters;
+    byte[] recordsJson;
+    JsonTreeRowRecordReader reader = null;
+
+    public AirtableRecordSet(final byte[] recordsJson,
+            final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory,
+            final AirtableRestService restService,
+            final AirtableGetRecordsParameters getRecordsParameters) {
+        this.recordReaderFactory = recordReaderFactory;
+        this.restService = restService;
+        this.getRecordsParameters = getRecordsParameters;
+        this.recordsJson = recordsJson;
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return recordReaderFactory.recordSchema;
+    }
+
+    @Override
+    public Record next() throws IOException {
+        if (reader == null) {
+            final ByteArrayInputStream inputStream = new ByteArrayInputStream(recordsJson);
+            try {
+                reader = recordReaderFactory.create(inputStream);
+            } catch (MalformedRecordException e) {
+                throw new IOException("Failed to create Airtable record reader", e);
+            }
+        }
+        final Record record;
+        try {
+            record = reader.nextRecord();
+        } catch (MalformedRecordException e) {
+            throw new IOException("Failed to read next Airtable record", e);
+        }
+
+        if (record != null) {
+            return record;
+        }
+
+        final Configuration configuration = Configuration.defaultConfiguration()
+                .addOptions(Option.DEFAULT_PATH_LEAF_TO_NULL);
+        final String offset = JsonPath.using(configuration).parse(new ByteArrayInputStream(recordsJson)).read("$.offset");

Review Comment:
   Using JSON Path to read this field is very in efficient because it requires reading the entire JSON structure into memory. This implementation should be refactored to avoid using the JSON Path library and instead use Jackson with stream-based reading.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/record/AirtableRecordSet.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.airtable.record;
+
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.Option;
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import org.apache.nifi.json.JsonTreeRowRecordReader;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceeded;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+public class AirtableRecordSet implements RecordSet, Closeable {
+
+    final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory;
+    final AirtableRestService restService;
+    final AirtableGetRecordsParameters getRecordsParameters;
+    byte[] recordsJson;
+    JsonTreeRowRecordReader reader = null;
+
+    public AirtableRecordSet(final byte[] recordsJson,
+            final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory,
+            final AirtableRestService restService,
+            final AirtableGetRecordsParameters getRecordsParameters) {
+        this.recordReaderFactory = recordReaderFactory;
+        this.restService = restService;
+        this.getRecordsParameters = getRecordsParameters;
+        this.recordsJson = recordsJson;
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return recordReaderFactory.recordSchema;
+    }
+
+    @Override
+    public Record next() throws IOException {
+        if (reader == null) {
+            final ByteArrayInputStream inputStream = new ByteArrayInputStream(recordsJson);
+            try {
+                reader = recordReaderFactory.create(inputStream);
+            } catch (MalformedRecordException e) {
+                throw new IOException("Failed to create Airtable record reader", e);
+            }
+        }
+        final Record record;
+        try {
+            record = reader.nextRecord();
+        } catch (MalformedRecordException e) {
+            throw new IOException("Failed to read next Airtable record", e);
+        }
+
+        if (record != null) {
+            return record;
+        }
+
+        final Configuration configuration = Configuration.defaultConfiguration()
+                .addOptions(Option.DEFAULT_PATH_LEAF_TO_NULL);
+        final String offset = JsonPath.using(configuration).parse(new ByteArrayInputStream(recordsJson)).read("$.offset");
+        if (offset != null) {
+            try {
+                recordsJson = restService.getRecords(getRecordsParameters.withOffset(offset));
+            } catch (RateLimitExceeded e) {
+                throw new IOException(e);

Review Comment:
   All exceptions should include a summary message, with should mention the rate limit error in this scenario.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>javax.ws.rs</groupId>
+            <artifactId>javax.ws.rs-api</artifactId>
+            <version>2.1.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-common</artifactId>
+            <scope>compile</scope>
+        </dependency>

Review Comment:
   Are these dependencies required? They should not be necessary for general HTTP client communication.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>

Review Comment:
   This dependency is provided and the version is defined in the `nifi-nar-bundles` Maven configuration, so these two lines can be removed.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>

Review Comment:
   This dependency should be marked as `provided`.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>

Review Comment:
   The `compile` scope is the default, so it should be removed from this and all dependencies.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceeded;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor API_TOKEN = new PropertyDescriptor.Builder()
+            .name("api-token")
+            .displayName("API Token")
+            .description("The REST API token to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per Flow File")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("output-batch-size")
+            .displayName("Output Batch Size")
+            .description("The number of output FlowFiles to queue before committing the process session. When set to zero, the session will be committed when all records"
+                    + " have been processed and the output FlowFiles are ready for transfer to the downstream relationship. For large result sets, this can cause a large burst of FlowFiles"
+                    + " to be transferred at the end of processor execution. If this property is set, then when the specified number of FlowFiles are ready for transfer, then the session will"
+                    + " be committed, thus releasing the FlowFiles to the downstream relationship.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
+            .name("metadata-strategy")
+            .displayName("Metadata Strategy")
+            .description("Strategy to use for fetching record schema. Currently only 'Use JSON Record Reader' is supported."
+                    + " When Airtable Metadata API becomes more stable it will be possible to fetch the record schema through it.")
+            .required(true)
+            .defaultValue(MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .allowableValues(MetadataStrategy.class)
+            .build();
+
+    static final PropertyDescriptor SCHEMA_READER = new PropertyDescriptor.Builder()
+            .name("schema-reader")
+            .displayName("Schema Reader")
+            .description("JsonTreeReader service to use for fetching the schema of records returned by the Airtable REST API")
+            .dependsOn(METADATA_STRATEGY, MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .identifiesControllerService(JsonRecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Service used for writing records returned by the Airtable REST API")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            BASE_ID,
+            TABLE_ID,
+            API_TOKEN,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            OUTPUT_BATCH_SIZE,
+            METADATA_STRATEGY,
+            SCHEMA_READER,
+            RECORD_WRITER,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiToken = context.getProperty(API_TOKEN).evaluateAttributeExpressions().getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider.getWebClientService(), apiUrl, apiToken, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final JsonRecordReaderFactory schemaRecordReaderFactory = context.getProperty(SCHEMA_READER).asControllerService(JsonRecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+        final Integer outputBatchSize = context.getProperty(OUTPUT_BATCH_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now().minusSeconds(QUERY_LAG_SECONDS).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final byte[] recordsJson;
+        try {
+            recordsJson = airtableRestService.getRecords(getRecordsParameters);
+        } catch (RateLimitExceeded e) {
+            context.yield();
+            throw new ProcessException(e);

Review Comment:
   This exception should include a message related to the Rate Limit exceeded.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>javax.ws.rs</groupId>
+            <artifactId>javax.ws.rs-api</artifactId>
+            <version>2.1.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-common</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock-record-utils</artifactId>
+            <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>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-proxy-configuration-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>

Review Comment:
   These dependencies do not appear to be used.



-- 
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] dam4rus commented on a diff in pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>javax.ws.rs</groupId>
+            <artifactId>javax.ws.rs-api</artifactId>
+            <version>2.1.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-common</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock-record-utils</artifactId>
+            <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>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-proxy-configuration-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>

Review Comment:
   These are required because `StandardWebClientServiceProvider` brings in `SSLContextService` and `ProxyConfigurationService` as dependency. I get `NoClassDefFound` otherwise. Is there a way to get rid of these dependencies in tests?



-- 
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] dam4rus commented on a diff in pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>javax.ws.rs</groupId>
+            <artifactId>javax.ws.rs-api</artifactId>
+            <version>2.1.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-common</artifactId>
+            <scope>compile</scope>
+        </dependency>

Review Comment:
   I use `UriBuilder` here
   
   https://github.com/apache/nifi/pull/6350/files#diff-e00c7213199741f61852a406a1d72569cd28df82ed2bdb0ae98ec4162327aa88R85
   
   `org.apache.nifi.web.client.api.HttpUriBuilder` doesn't seem to support parsing URI strings which would be convenient here. I would require parsing the URI and pass scheme, host and path to `org.apache.nifi.web.client.api.HttpUriBuilder` one by one. Is there a better way?



-- 
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] exceptionfactory commented on a diff in pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/record/AirtableRecordSet.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.airtable.record;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.nifi.json.JsonTreeRowRecordReader;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+
+public class AirtableRecordSet implements RecordSet, Closeable {
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory();
+
+    final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory;
+    final AirtableRestService restService;
+    final AirtableGetRecordsParameters getRecordsParameters;
+    byte[] recordsJson;
+    JsonTreeRowRecordReader reader = null;
+
+    public AirtableRecordSet(final byte[] recordsJson,
+            final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory,
+            final AirtableRestService restService,
+            final AirtableGetRecordsParameters getRecordsParameters) {
+        this.recordReaderFactory = recordReaderFactory;
+        this.restService = restService;
+        this.getRecordsParameters = getRecordsParameters;
+        this.recordsJson = recordsJson;
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return recordReaderFactory.recordSchema;
+    }
+
+    @Override
+    public Record next() throws IOException {
+        if (reader == null) {
+            final ByteArrayInputStream inputStream = new ByteArrayInputStream(recordsJson);
+            try {
+                reader = recordReaderFactory.create(inputStream);
+            } catch (MalformedRecordException e) {
+                throw new IOException("Failed to create Airtable record reader", e);
+            }
+        }
+        final Record record;
+        try {
+            record = reader.nextRecord();
+        } catch (MalformedRecordException e) {
+            throw new IOException("Failed to read next Airtable record", e);
+        }
+
+        if (record != null) {
+            return record;
+        }
+
+        final Optional<String> offset = readOffsetFromJson();
+        if (offset.isPresent()) {
+            try {
+                recordsJson = restService.getRecords(getRecordsParameters.withOffset(offset.get()));
+            } catch (RateLimitExceededException e) {
+                throw new IOException("REST API rate limit exceeded while fetching Airtable records", e);
+            }
+            reader = null;
+            return next();
+        }
+
+        return null;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (reader != null) {
+            reader.close();
+        }
+    }
+
+    private Optional<String> readOffsetFromJson() throws IOException {
+        try (final JsonParser jsonParser = JSON_FACTORY.createParser(recordsJson)) {

Review Comment:
   This approach is an improvement over using JSON Path, thanks for making the adjustments. However, it still requires parsing the response JSON multiple times. A custom Record Reader seems like it will be necessary to read records, and also read and store the offset while processing.



##########
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java:
##########
@@ -67,7 +67,7 @@
         + "a field that is not present in the schema, that field will be skipped. "
         + "See the Usage of the Controller Service for more information and examples.")
 @SeeAlso(JsonPathReader.class)
-public class JsonTreeReader extends SchemaRegistryService implements RecordReaderFactory {
+public class JsonTreeReader extends SchemaRegistryService implements JsonRecordReaderFactory {

Review Comment:
   As mentioned on the extended interface, this change appears unnecessary.



##########
nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/JsonRecordReaderFactory.java:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.serialization;
+
+public interface JsonRecordReaderFactory extends RecordReaderFactory {
+
+}

Review Comment:
   Is there a particular reason for this extended interface? A format-specific interface does not belong in this module, and it seems like it should be removed.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableRestService.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.airtable.service;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import javax.ws.rs.core.UriBuilder;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.Range;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.WebClientService;
+
+public class AirtableRestService {
+
+    public static final String API_V0_BASE_URL = "https://api.airtable.com/v0";
+
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Range<Integer> SUCCESSFUL_RESPONSE_RANGE = Range.between(200, 299);
+
+    private final WebClientService webClientService;
+    private final String apiUrl;
+    private final String apiKey;
+    private final String baseId;
+    private final String tableId;
+
+    public AirtableRestService(final WebClientService webClientService,
+            final String apiUrl,
+            final String apiKey,
+            final String baseId,
+            final String tableId) {
+        this.webClientService = webClientService;
+        this.apiUrl = apiUrl;
+        this.apiKey = apiKey;
+        this.baseId = baseId;
+        this.tableId = tableId;
+    }
+
+    public byte[] getRecords(final AirtableGetRecordsParameters filter) throws RateLimitExceededException {
+        final URI uri = buildUri(filter);
+        try (final HttpResponseEntity response = webClientService.get()
+                .uri(uri)
+                .header("Authorization", "Bearer " + apiKey)
+                .retrieve()) {
+
+            if (SUCCESSFUL_RESPONSE_RANGE.contains(response.statusCode())) {
+                return IOUtils.toByteArray(response.body());
+            }
+            if (response.statusCode() == TOO_MANY_REQUESTS) {
+                throw new RateLimitExceededException();
+            }
+            final StringBuilder exceptionMessageBuilder = new StringBuilder("Invalid response. Code: " + response.statusCode());
+            final String bodyText = IOUtils.toString(response.body(), StandardCharsets.UTF_8);
+            if (bodyText != null) {
+                exceptionMessageBuilder.append(" Body: ").append(bodyText);
+            }
+
+            throw new ProcessException(exceptionMessageBuilder.toString());
+        } catch (IOException e) {
+            throw new ProcessException(String.format("Airtable HTTP request failed [%s]", uri), e);
+        }
+    }
+
+    public static class RateLimitExceededException extends Exception {

Review Comment:
   It seems like this would be better defined as a subclass of `RuntimeException`. Making this a separate class, as opposed to a public static member, also seems like a better approach.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per Flow File")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("output-batch-size")
+            .displayName("Output Batch Size")
+            .description("The number of output FlowFiles to queue before committing the process session. When set to zero, the session will be committed when all records"
+                    + " have been processed and the output FlowFiles are ready for transfer to the downstream relationship. For large result sets, this can cause a large burst of FlowFiles"
+                    + " to be transferred at the end of processor execution. If this property is set, then when the specified number of FlowFiles are ready for transfer, then the session will"
+                    + " be committed, thus releasing the FlowFiles to the downstream relationship.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
+            .name("metadata-strategy")
+            .displayName("Metadata Strategy")
+            .description("Strategy to use for fetching record schema. Currently only 'Use JSON Record Reader' is supported."
+                    + " When Airtable Metadata API becomes more stable it will be possible to fetch the record schema through it.")
+            .required(true)
+            .defaultValue(MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .allowableValues(MetadataStrategy.class)
+            .build();
+
+    static final PropertyDescriptor SCHEMA_READER = new PropertyDescriptor.Builder()
+            .name("schema-reader")
+            .displayName("Schema Reader")
+            .description("JsonTreeReader service to use for fetching the schema of records returned by the Airtable REST API")
+            .dependsOn(METADATA_STRATEGY, MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .identifiesControllerService(JsonRecordReaderFactory.class)

Review Comment:
   Although this interface allows narrowing the supported type, it introduces a format-specific interface to the API, which is not ideal.
   
   Is this necessary given the custom `AirtableJsonTreeRowRecordReaderFactory`?



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/record/AirtableJsonTreeRowRecordReaderFactory.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.airtable.record;
+
+import java.io.IOException;
+import java.io.InputStream;
+import org.apache.nifi.json.JsonTreeRowRecordReader;
+import org.apache.nifi.json.SchemaApplicationStrategy;
+import org.apache.nifi.json.StartingFieldStrategy;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+public class AirtableJsonTreeRowRecordReaderFactory {
+
+    private static final String STARTING_FIELD_NAME = "records";
+    private static final String DATE_FORMAT = "yyyy-MM-dd";
+    private static final String TIME_FORMAT = "HH:mm:ss.SSSX";
+    private static final String DATE_TIME_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSZZZZ";

Review Comment:
   Do these formats apply to all records in Airtable? It would be helpful to provide a short comment with a link to the standard if available.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/test/java/org/apache/nifi/processors/airtable/QueryAirtableTableIT.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.airtable;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import okhttp3.HttpUrl;
+import okhttp3.mockwebserver.MockResponse;
+import okhttp3.mockwebserver.MockWebServer;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.json.SchemaApplicationStrategy;
+import org.apache.nifi.json.StartingFieldStrategy;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+import org.apache.nifi.web.client.provider.service.StandardWebClientServiceProvider;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class QueryAirtableTableIT {
+
+    public static final String RECORDS_JSON_BODY = "{\"records\":[{"
+            + "\"id\":\"recabcdefghijklmn\","
+            + "\"createdTime\":\"1970-00-01T00:00:00.000Z\","
+            + "\"fields\":{\"foo\":\"bar\"}}]}";
+    public static final String RECORDS_WITH_OFFSET_JSON_BODY = "{\"records\":[{"
+            + "\"id\":\"recabcdefghijklmn\","
+            + "\"createdTime\":\"1970-00-01T00:00:00.000Z\","
+            + "\"fields\":{\"foo\":\"bar\"}}],"
+            + "\"offset\":\"ofsabcdefghijklmn\"}";
+    public static final String EXPECTED_RECORD_CONTENT =
+            "\"recabcdefghijklmn\",\"1970-00-01T00:00:00.000Z\",\"MapRecord[{foo=bar}]\"\n";
+
+    private TestRunner runner;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        final Processor queryAirtableTable = new QueryAirtableTable();
+
+        runner = TestRunners.newTestRunner(queryAirtableTable);
+
+        final RecordReaderFactory schemaReader = new JsonTreeReader();
+        runner.addControllerService("reader", schemaReader);
+        runner.setProperty(schemaReader, JsonTreeReader.STARTING_FIELD_NAME, "records");
+        runner.setProperty(schemaReader, JsonTreeReader.STARTING_FIELD_STRATEGY, StartingFieldStrategy.NESTED_FIELD.getValue());
+        runner.setProperty(schemaReader, JsonTreeReader.SCHEMA_APPLICATION_STRATEGY, SchemaApplicationStrategy.SELECTED_PART.getValue());
+        runner.enableControllerService(schemaReader);
+
+        final RecordSetWriterFactory writer = new MockRecordWriter();
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        final WebClientServiceProvider webClientServiceProvider = new StandardWebClientServiceProvider();
+        runner.addControllerService("webClientService", webClientServiceProvider);
+        runner.enableControllerService(webClientServiceProvider);
+
+        runner.setProperty(QueryAirtableTable.API_KEY, "???");
+        runner.setProperty(QueryAirtableTable.BASE_ID, "appabcdefghijklmn");
+        runner.setProperty(QueryAirtableTable.TABLE_ID, "tblabcdefghijklmn");

Review Comment:
   Recommend using a placeholder word, or number, instead of random characters. Either `UUID.randomUUID().toString()` or just `BASE_ID` and `TABLE_ID`.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,118 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-processors</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-api</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>javax.ws.rs</groupId>
+            <artifactId>javax.ws.rs-api</artifactId>
+            <version>2.1.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.glassfish.jersey.core</groupId>
+            <artifactId>jersey-common</artifactId>
+            <scope>compile</scope>
+        </dependency>

Review Comment:
   Thanks for the reference. The standard `java.net.URI.create()` method can parse a URI string, which seems like it should work in this case, and avoid introducing these additional dependencies. If necessary, it may be worth extending the `HttpUriBuilder`, but either approach would be better than introducing these dependencies just for parsing a URI.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/test/java/org/apache/nifi/processors/airtable/QueryAirtableTableIT.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.airtable;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import okhttp3.HttpUrl;
+import okhttp3.mockwebserver.MockResponse;
+import okhttp3.mockwebserver.MockWebServer;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.json.SchemaApplicationStrategy;
+import org.apache.nifi.json.StartingFieldStrategy;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+import org.apache.nifi.web.client.provider.service.StandardWebClientServiceProvider;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class QueryAirtableTableIT {

Review Comment:
   It looks like this test can run as a unit test, which would provide validation as part of automated builds.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per Flow File")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("output-batch-size")
+            .displayName("Output Batch Size")
+            .description("The number of output FlowFiles to queue before committing the process session. When set to zero, the session will be committed when all records"
+                    + " have been processed and the output FlowFiles are ready for transfer to the downstream relationship. For large result sets, this can cause a large burst of FlowFiles"
+                    + " to be transferred at the end of processor execution. If this property is set, then when the specified number of FlowFiles are ready for transfer, then the session will"
+                    + " be committed, thus releasing the FlowFiles to the downstream relationship.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
+            .name("metadata-strategy")
+            .displayName("Metadata Strategy")
+            .description("Strategy to use for fetching record schema. Currently only 'Use JSON Record Reader' is supported."
+                    + " When Airtable Metadata API becomes more stable it will be possible to fetch the record schema through it.")
+            .required(true)
+            .defaultValue(MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .allowableValues(MetadataStrategy.class)
+            .build();
+
+    static final PropertyDescriptor SCHEMA_READER = new PropertyDescriptor.Builder()
+            .name("schema-reader")
+            .displayName("Schema Reader")
+            .description("JsonTreeReader service to use for fetching the schema of records returned by the Airtable REST API")
+            .dependsOn(METADATA_STRATEGY, MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .identifiesControllerService(JsonRecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Service used for writing records returned by the Airtable REST API")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            OUTPUT_BATCH_SIZE,
+            METADATA_STRATEGY,
+            SCHEMA_READER,
+            RECORD_WRITER,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider.getWebClientService(), apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final JsonRecordReaderFactory schemaRecordReaderFactory = context.getProperty(SCHEMA_READER).asControllerService(JsonRecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+        final Integer outputBatchSize = context.getProperty(OUTPUT_BATCH_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now().minusSeconds(QUERY_LAG_SECONDS).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final byte[] recordsJson;
+        try {
+            recordsJson = airtableRestService.getRecords(getRecordsParameters);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("REST API rate limit exceeded while fetching initial Airtable record set", e);
+        }
+
+        final FlowFile flowFile = session.create();
+        final Map<String, String> originalAttributes = flowFile.getAttributes();
+
+        final RecordSchema recordSchema;
+        final RecordSchema writerSchema;
+        try {
+            final ByteArrayInputStream recordsStream = new ByteArrayInputStream(recordsJson);
+            recordSchema = schemaRecordReaderFactory.createRecordReader(flowFile, recordsStream, getLogger()).getSchema();
+            writerSchema = writerFactory.getSchema(originalAttributes, recordSchema);
+        } catch (MalformedRecordException | IOException | SchemaNotFoundException e) {
+            throw new ProcessException("Couldn't get record schema", e);
+        }
+
+        session.remove(flowFile);
+
+        final List<FlowFile> flowFiles = new ArrayList<>();
+        int totalRecordCount = 0;
+        final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory = new AirtableJsonTreeRowRecordReaderFactory(getLogger(), recordSchema);
+        try (final AirtableRecordSet airtableRecordSet = new AirtableRecordSet(recordsJson, recordReaderFactory, airtableRestService, getRecordsParameters)) {
+            while (true) {
+                final AtomicInteger recordCountHolder = new AtomicInteger();
+                final Map<String, String> attributes = new HashMap<>();
+                FlowFile flowFileToAdd = session.create();
+                flowFileToAdd = session.write(flowFileToAdd, out -> {
+                    try (final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writerSchema, out, originalAttributes)) {
+                        final RecordSet recordSet = (maxRecordsPerFlowFile > 0 ? airtableRecordSet.limit(maxRecordsPerFlowFile) : airtableRecordSet);
+                        final WriteResult writeResult = writer.write(recordSet);
+
+                        attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+                        attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
+                        attributes.putAll(writeResult.getAttributes());
+
+                        recordCountHolder.set(writeResult.getRecordCount());
+                    } catch (final IOException e) {
+                        final Throwable cause = e.getCause();
+                        if (cause instanceof RateLimitExceededException) {
+                            context.yield();
+                            throw new ProcessException("REST API rate limit exceeded while reading Airtable records", cause);
+                        }
+                        throw new ProcessException("Couldn't read records from input", e);
+                    } catch (final SchemaNotFoundException e) {
+                        throw new ProcessException("Couldn't read records from input", e);
+                    }
+                });
+
+                flowFileToAdd = session.putAllAttributes(flowFileToAdd, attributes);
+
+                final int recordCount = recordCountHolder.get();
+                if (recordCount > 0) {
+                    totalRecordCount += recordCount;
+                    flowFiles.add(flowFileToAdd);
+                    if (outputBatchSize > 0 && flowFiles.size() == outputBatchSize) {
+                        transferFlowFiles(flowFiles, session, totalRecordCount);
+                        flowFiles.clear();
+                    }
+                    continue;
+                }
+
+                session.remove(flowFileToAdd);
+
+                if (maxRecordsPerFlowFile > 0 && outputBatchSize == 0) {
+                    fragmentFlowFiles(session, flowFiles);
+                }
+
+                transferFlowFiles(flowFiles, session, totalRecordCount);
+                break;
+            }
+        } catch (final IOException e) {
+            throw new ProcessException("Couldn't read records from input", e);
+        }
+
+        if (totalRecordCount == 0) {
+            return;
+        }
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to update cluster state", e);
+        }
+    }
+
+    private AirtableGetRecordsParameters buildGetRecordsParameters(final ProcessContext context,
+            final String lastRecordFetchTime,
+            final String nowDateTimeString) {
+        final String fieldsProperty = context.getProperty(FIELDS).evaluateAttributeExpressions().getValue();
+        final String customFilter = context.getProperty(CUSTOM_FILTER).evaluateAttributeExpressions().getValue();
+        final Integer pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final AirtableGetRecordsParameters.Builder getRecordsParametersBuilder = new AirtableGetRecordsParameters.Builder();
+        if (lastRecordFetchTime != null) {
+            getRecordsParametersBuilder
+                    .modifiedAfter(lastRecordFetchTime)
+                    .modifiedBefore(nowDateTimeString);
+        }
+        if (fieldsProperty != null) {
+            getRecordsParametersBuilder.fields(Arrays.stream(fieldsProperty.split(",")).map(String::trim).collect(Collectors.toList()));
+        }
+        getRecordsParametersBuilder.customFilter(customFilter);
+        if (pageSize != null && pageSize > 0) {
+            getRecordsParametersBuilder.pageSize(pageSize);
+        }
+
+        return getRecordsParametersBuilder.build();
+    }
+
+    private void fragmentFlowFiles(final ProcessSession session, final List<FlowFile> flowFiles) {
+        final String fragmentIdentifier = UUID.randomUUID().toString();
+        for (int i = 0; i < flowFiles.size(); i++) {
+            final Map<String, String> fragmentAttributes = new HashMap<>();
+            fragmentAttributes.put(FRAGMENT_ID.key(), fragmentIdentifier);
+            fragmentAttributes.put(FRAGMENT_INDEX.key(), String.valueOf(i));
+            fragmentAttributes.put(FRAGMENT_COUNT.key(), String.valueOf(flowFiles.size()));
+
+            flowFiles.set(i, session.putAllAttributes(flowFiles.get(i), fragmentAttributes));
+        }
+    }
+
+    private void transferFlowFiles(final List<FlowFile> flowFiles, final ProcessSession session, final int totalRecordCount) {
+        session.transfer(flowFiles, REL_SUCCESS);
+        session.adjustCounter("Records Processed", totalRecordCount, false);
+        final String flowFilesAsString = flowFiles.stream().map(FlowFile::toString).collect(Collectors.joining(", ", "[", "]"));
+        getLogger().info("Successfully written {} records for flow files {}", totalRecordCount, flowFilesAsString);

Review Comment:
   Recommend adjusting the wording:
   ```suggestion
           getLogger().info("Transferred FlowFiles [{}] Records [{}]", flowFilesAsString, totalRecordCount);
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableGetRecordsParameters.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.airtable.service;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+
+public class AirtableGetRecordsParameters {
+
+    private final List<String> fields;
+    private final Optional<String> modifiedAfter;
+    private final Optional<String> modifiedBefore;
+    private final Optional<String> customFilter;
+    private final Optional<String> offset;
+    private final OptionalInt pageSize;
+
+    public AirtableGetRecordsParameters(final List<String> fields,
+            final Optional<String> modifiedAfter,
+            final Optional<String> modifiedBefore,
+            final Optional<String> customFilter,
+            final Optional<String> offset,
+            final OptionalInt pageSize) {
+        this.fields = Objects.requireNonNull(fields);
+        this.modifiedAfter = modifiedAfter;
+        this.modifiedBefore = modifiedBefore;
+        this.customFilter = customFilter;
+        this.offset = offset;
+        this.pageSize = pageSize;
+    }
+
+    public List<String> getFields() {
+        return fields;
+    }
+
+    public Optional<String> getModifiedAfter() {
+        return modifiedAfter;
+    }
+
+    public Optional<String> getModifiedBefore() {
+        return modifiedBefore;
+    }
+
+    public Optional<String> getCustomFilter() {
+        return customFilter;
+    }
+
+    public Optional<String> getOffset() {
+        return offset;
+    }
+
+    public OptionalInt getPageSize() {
+        return pageSize;
+    }
+
+    public AirtableGetRecordsParameters withOffset(final String offset) {
+        return new AirtableGetRecordsParameters(fields, modifiedAfter, modifiedBefore, customFilter, Optional.of(offset), pageSize);
+    }
+
+    public static class Builder {
+        private List<String> fields = null;
+        private String modifiedAfter = null;
+        private String modifiedBefore = null;
+        private String customFilter = null;
+        private String offset = null;

Review Comment:
   Declaring `null` should be unnecessary, since it is the default value.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")

Review Comment:
   Recommend calling this property `Table ID`, since the description indicates that the ID or name is allowed.



##########
nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-mock-record-utils/src/main/java/org/apache/nifi/serialization/record/MockRecordParser.java:
##########
@@ -33,7 +34,7 @@
 import java.util.List;
 import java.util.Map;
 
-public class MockRecordParser extends AbstractControllerService implements RecordReaderFactory {
+public class MockRecordParser extends AbstractControllerService implements JsonRecordReaderFactory, RecordReaderFactory {

Review Comment:
   The `MockRecordReader` does not have any JSON-specific elements, so this change should be reverted.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/test/java/org/apache/nifi/processors/airtable/QueryAirtableTableIT.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.airtable;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import okhttp3.HttpUrl;
+import okhttp3.mockwebserver.MockResponse;
+import okhttp3.mockwebserver.MockWebServer;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.json.SchemaApplicationStrategy;
+import org.apache.nifi.json.StartingFieldStrategy;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+import org.apache.nifi.web.client.provider.service.StandardWebClientServiceProvider;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class QueryAirtableTableIT {
+
+    public static final String RECORDS_JSON_BODY = "{\"records\":[{"
+            + "\"id\":\"recabcdefghijklmn\","
+            + "\"createdTime\":\"1970-00-01T00:00:00.000Z\","
+            + "\"fields\":{\"foo\":\"bar\"}}]}";
+    public static final String RECORDS_WITH_OFFSET_JSON_BODY = "{\"records\":[{"
+            + "\"id\":\"recabcdefghijklmn\","
+            + "\"createdTime\":\"1970-00-01T00:00:00.000Z\","
+            + "\"fields\":{\"foo\":\"bar\"}}],"
+            + "\"offset\":\"ofsabcdefghijklmn\"}";
+    public static final String EXPECTED_RECORD_CONTENT =
+            "\"recabcdefghijklmn\",\"1970-00-01T00:00:00.000Z\",\"MapRecord[{foo=bar}]\"\n";
+
+    private TestRunner runner;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        final Processor queryAirtableTable = new QueryAirtableTable();
+
+        runner = TestRunners.newTestRunner(queryAirtableTable);
+
+        final RecordReaderFactory schemaReader = new JsonTreeReader();
+        runner.addControllerService("reader", schemaReader);
+        runner.setProperty(schemaReader, JsonTreeReader.STARTING_FIELD_NAME, "records");
+        runner.setProperty(schemaReader, JsonTreeReader.STARTING_FIELD_STRATEGY, StartingFieldStrategy.NESTED_FIELD.getValue());
+        runner.setProperty(schemaReader, JsonTreeReader.SCHEMA_APPLICATION_STRATEGY, SchemaApplicationStrategy.SELECTED_PART.getValue());
+        runner.enableControllerService(schemaReader);
+
+        final RecordSetWriterFactory writer = new MockRecordWriter();
+        runner.addControllerService("writer", writer);
+        runner.enableControllerService(writer);
+
+        final WebClientServiceProvider webClientServiceProvider = new StandardWebClientServiceProvider();
+        runner.addControllerService("webClientService", webClientServiceProvider);
+        runner.enableControllerService(webClientServiceProvider);
+
+        runner.setProperty(QueryAirtableTable.API_KEY, "???");
+        runner.setProperty(QueryAirtableTable.BASE_ID, "appabcdefghijklmn");
+        runner.setProperty(QueryAirtableTable.TABLE_ID, "tblabcdefghijklmn");
+        runner.setProperty(QueryAirtableTable.SCHEMA_READER, schemaReader.getIdentifier());
+        runner.setProperty(QueryAirtableTable.RECORD_WRITER, writer.getIdentifier());
+        runner.setProperty(QueryAirtableTable.WEB_CLIENT_SERVICE_PROVIDER, webClientServiceProvider.getIdentifier());
+
+    }
+
+    @AfterEach
+    void tearDown() {
+        runner.shutdown();
+    }
+
+    @Test
+    void retrievesAndWritesRecords() throws Exception {
+        try (final MockWebServer server = new MockWebServer()) {
+            server.enqueue(new MockResponse().setBody(RECORDS_JSON_BODY));
+
+            server.start();
+            final HttpUrl httpUrl = server.url("/v0/airtable");
+
+            runner.setProperty(QueryAirtableTable.API_URL, httpUrl.toString());
+            runner.run();
+
+            final List<MockFlowFile> results = runner.getFlowFilesForRelationship(QueryAirtableTable.REL_SUCCESS);
+            assertEquals(1, results.size());
+            final MockFlowFile flowFile = results.get(0);
+            assertEquals("1", flowFile.getAttribute("record.count"));
+            final String content = flowFile.getContent();
+            assertEquals(EXPECTED_RECORD_CONTENT, content);
+        }
+    }
+
+    @Test
+    void retrievesAndWritesPagedRecords() throws Exception {
+        try (final MockWebServer server = new MockWebServer()) {
+            server.enqueue(new MockResponse().setBody(RECORDS_WITH_OFFSET_JSON_BODY));
+            server.enqueue(new MockResponse().setBody(RECORDS_JSON_BODY));
+
+            server.start();
+            final HttpUrl httpUrl = server.url("/v0/airtable");
+
+            runner.setProperty(QueryAirtableTable.API_URL, httpUrl.toString());
+            runner.run();
+
+            final List<MockFlowFile> results = runner.getFlowFilesForRelationship(QueryAirtableTable.REL_SUCCESS);
+            assertEquals(1, results.size());
+            final MockFlowFile flowFile = results.get(0);
+            assertEquals("2", flowFile.getAttribute("record.count"));
+            final String content = flowFile.getContent();
+            assertEquals(EXPECTED_RECORD_CONTENT + EXPECTED_RECORD_CONTENT, content);
+        }
+    }
+
+    @Test
+    void retrievesAndWritesPagedRecordsInMultipleFlowFiles() throws Exception {
+        try (final MockWebServer server = new MockWebServer()) {
+            server.enqueue(new MockResponse().setBody(RECORDS_WITH_OFFSET_JSON_BODY));
+            server.enqueue(new MockResponse().setBody(RECORDS_JSON_BODY));
+
+            server.start();
+            final HttpUrl httpUrl = server.url("/v0/airtable");

Review Comment:
   Recommend declaring `/v0/airtable` as a static member variable and reusing across all test methods.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per Flow File")

Review Comment:
   ```suggestion
               .displayName("Max Records Per FlowFile")
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per Flow File")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("output-batch-size")
+            .displayName("Output Batch Size")
+            .description("The number of output FlowFiles to queue before committing the process session. When set to zero, the session will be committed when all records"
+                    + " have been processed and the output FlowFiles are ready for transfer to the downstream relationship. For large result sets, this can cause a large burst of FlowFiles"
+                    + " to be transferred at the end of processor execution. If this property is set, then when the specified number of FlowFiles are ready for transfer, then the session will"
+                    + " be committed, thus releasing the FlowFiles to the downstream relationship.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
+            .name("metadata-strategy")
+            .displayName("Metadata Strategy")
+            .description("Strategy to use for fetching record schema. Currently only 'Use JSON Record Reader' is supported."
+                    + " When Airtable Metadata API becomes more stable it will be possible to fetch the record schema through it.")
+            .required(true)
+            .defaultValue(MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .allowableValues(MetadataStrategy.class)
+            .build();
+
+    static final PropertyDescriptor SCHEMA_READER = new PropertyDescriptor.Builder()
+            .name("schema-reader")
+            .displayName("Schema Reader")
+            .description("JsonTreeReader service to use for fetching the schema of records returned by the Airtable REST API")
+            .dependsOn(METADATA_STRATEGY, MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .identifiesControllerService(JsonRecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Service used for writing records returned by the Airtable REST API")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            OUTPUT_BATCH_SIZE,
+            METADATA_STRATEGY,
+            SCHEMA_READER,
+            RECORD_WRITER,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider.getWebClientService(), apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final JsonRecordReaderFactory schemaRecordReaderFactory = context.getProperty(SCHEMA_READER).asControllerService(JsonRecordReaderFactory.class);
+        final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+        final Integer outputBatchSize = context.getProperty(OUTPUT_BATCH_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now().minusSeconds(QUERY_LAG_SECONDS).format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final byte[] recordsJson;
+        try {
+            recordsJson = airtableRestService.getRecords(getRecordsParameters);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("REST API rate limit exceeded while fetching initial Airtable record set", e);
+        }
+
+        final FlowFile flowFile = session.create();
+        final Map<String, String> originalAttributes = flowFile.getAttributes();
+
+        final RecordSchema recordSchema;
+        final RecordSchema writerSchema;
+        try {
+            final ByteArrayInputStream recordsStream = new ByteArrayInputStream(recordsJson);
+            recordSchema = schemaRecordReaderFactory.createRecordReader(flowFile, recordsStream, getLogger()).getSchema();
+            writerSchema = writerFactory.getSchema(originalAttributes, recordSchema);
+        } catch (MalformedRecordException | IOException | SchemaNotFoundException e) {
+            throw new ProcessException("Couldn't get record schema", e);
+        }
+
+        session.remove(flowFile);
+
+        final List<FlowFile> flowFiles = new ArrayList<>();
+        int totalRecordCount = 0;
+        final AirtableJsonTreeRowRecordReaderFactory recordReaderFactory = new AirtableJsonTreeRowRecordReaderFactory(getLogger(), recordSchema);
+        try (final AirtableRecordSet airtableRecordSet = new AirtableRecordSet(recordsJson, recordReaderFactory, airtableRestService, getRecordsParameters)) {

Review Comment:
   It looks like this approach needs to be re-evaluated. Reading the JSON byte array to determine the schema, and then reading it again to process the records is not ideal for memory or performance. If it is not possible to infer the schema after parsing the first record using existing components, then it would be worth evaluating a new implementation that is capable of performing both actions without requiring multiple reads. This approach should also allow reading the JSON as a stream, instead of a byte array.



-- 
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] exceptionfactory commented on a diff in pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")

Review Comment:
   Is 100 the maximum number supported by the Airtable service?



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableRestService.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.airtable.service;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import javax.ws.rs.core.UriBuilder;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.WebClientService;
+
+public class AirtableRestService {
+
+    public static final String API_V0_BASE_URL = "https://api.airtable.com/v0";
+
+    private final WebClientService webClientService;
+    private final String apiUrl;
+    private final String apiToken;
+    private final String baseId;
+    private final String tableId;
+
+    public AirtableRestService(final WebClientService webClientService,
+            final String apiUrl,
+            final String apiToken,
+            final String baseId,
+            final String tableId) {
+        this.webClientService = webClientService;
+        this.apiUrl = apiUrl;
+        this.apiToken = apiToken;
+        this.baseId = baseId;
+        this.tableId = tableId;
+    }
+
+    public byte[] getRecords(final AirtableGetRecordsParameters filter) throws RateLimitExceeded {
+        final URI uri = buildUri(filter);
+        try (final HttpResponseEntity response = webClientService.get()
+                .uri(uri)
+                .header("Authorization", "Bearer " + apiToken)
+                .retrieve()) {
+
+            if (response.statusCode() >= 200 && response.statusCode() < 300) {
+                return IOUtils.toByteArray(response.body());
+            }
+            if (response.statusCode() == 429) {
+                throw new RateLimitExceeded();
+            }
+            final StringBuilder exceptionMessageBuilder = new StringBuilder("Invalid response. Code: " + response.statusCode());
+            final String bodyText = IOUtils.toString(response.body(), StandardCharsets.UTF_8);
+            if (bodyText != null) {
+                exceptionMessageBuilder.append(" Body: ").append(bodyText);
+            }
+
+            throw new ProcessException(exceptionMessageBuilder.toString());
+        } catch (IOException e) {
+            throw new ProcessException(String.format("Airtable HTTP request failed [%s]", uri), e);
+        }
+    }
+
+    public static class RateLimitExceeded extends Exception {

Review Comment:
   It is quite common to add -Exception suffix for exception classes: `RateLimitExceededException`



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableRestService.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.airtable.service;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import javax.ws.rs.core.UriBuilder;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.WebClientService;
+
+public class AirtableRestService {
+
+    public static final String API_V0_BASE_URL = "https://api.airtable.com/v0";
+
+    private final WebClientService webClientService;
+    private final String apiUrl;
+    private final String apiToken;
+    private final String baseId;
+    private final String tableId;
+
+    public AirtableRestService(final WebClientService webClientService,
+            final String apiUrl,
+            final String apiToken,
+            final String baseId,
+            final String tableId) {
+        this.webClientService = webClientService;
+        this.apiUrl = apiUrl;
+        this.apiToken = apiToken;
+        this.baseId = baseId;
+        this.tableId = tableId;
+    }
+
+    public byte[] getRecords(final AirtableGetRecordsParameters filter) throws RateLimitExceeded {
+        final URI uri = buildUri(filter);
+        try (final HttpResponseEntity response = webClientService.get()
+                .uri(uri)
+                .header("Authorization", "Bearer " + apiToken)
+                .retrieve()) {
+
+            if (response.statusCode() >= 200 && response.statusCode() < 300) {
+                return IOUtils.toByteArray(response.body());
+            }
+            if (response.statusCode() == 429) {
+                throw new RateLimitExceeded();
+            }
+            final StringBuilder exceptionMessageBuilder = new StringBuilder("Invalid response. Code: " + response.statusCode());
+            final String bodyText = IOUtils.toString(response.body(), StandardCharsets.UTF_8);
+            if (bodyText != null) {
+                exceptionMessageBuilder.append(" Body: ").append(bodyText);
+            }
+
+            throw new ProcessException(exceptionMessageBuilder.toString());
+        } catch (IOException e) {
+            throw new ProcessException(String.format("Airtable HTTP request failed [%s]", uri), e);
+        }
+    }
+
+    public static class RateLimitExceeded extends Exception {
+        public RateLimitExceeded() {
+            super("Airtable REST API rate limit exceeded");
+        }
+    }
+
+    private URI buildUri(AirtableGetRecordsParameters filter) {
+        final UriBuilder uriBuilder = UriBuilder.fromUri(apiUrl).path(baseId).path(tableId);
+
+        for (final String field : filter.getFields()) {
+            uriBuilder.queryParam("fields[]", field);
+        }
+
+        final List<String> filtersByFormula = new ArrayList<>();
+        filter.getFilterByFormula()
+                .ifPresent(filtersByFormula::add);
+        filter.getModifiedAfter()
+                .map(modifiedAfter -> "IS_AFTER(LAST_MODIFIED_TIME(),DATETIME_PARSE(\"" + modifiedAfter + "\"))")
+                .ifPresent(filtersByFormula::add);
+        filter.getModifiedBefore()
+                .map(modifiedBefore -> "IS_BEFORE(LAST_MODIFIED_TIME(),DATETIME_PARSE(\"" + modifiedBefore + "\"))")
+                .ifPresent(filtersByFormula::add);
+        if (!filtersByFormula.isEmpty()) {
+            uriBuilder.queryParam("filterByFormula", "AND(" + String.join(",", filtersByFormula) + ")");
+        }
+        filter.getOffset().ifPresent(offset -> uriBuilder.queryParam("offset", offset));
+        filter.getPageSize().ifPresent(pageSize -> uriBuilder.queryParam("pageSize", pageSize));
+
+        return uriBuilder.build();
+    }
+}

Review Comment:
   It is not easy to follow here what are `filter` / `filterByFormula` / `filtersByFormula`, so I suggest renaming these variables:
   - `filter` => `getRecordsParameters`
   - `AirtableGetRecordsParameters.filterByFormula` => `AirtableGetRecordsParameters.customFilter`
   - `filtersByFormula` => `filters`
   



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceeded;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor API_TOKEN = new PropertyDescriptor.Builder()
+            .name("api-token")
+            .displayName("API Token")
+            .description("The REST API token to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per Flow File")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor OUTPUT_BATCH_SIZE = new PropertyDescriptor.Builder()
+            .name("output-batch-size")
+            .displayName("Output Batch Size")
+            .description("The number of output FlowFiles to queue before committing the process session. When set to zero, the session will be committed when all records"
+                    + " have been processed and the output FlowFiles are ready for transfer to the downstream relationship. For large result sets, this can cause a large burst of FlowFiles"
+                    + " to be transferred at the end of processor execution. If this property is set, then when the specified number of FlowFiles are ready for transfer, then the session will"
+                    + " be committed, thus releasing the FlowFiles to the downstream relationship.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor METADATA_STRATEGY = new PropertyDescriptor.Builder()
+            .name("metadata-strategy")
+            .displayName("Metadata Strategy")
+            .description("Strategy to use for fetching record schema. Currently only 'Use JSON Record Reader' is supported."
+                    + " When Airtable Metadata API becomes more stable it will be possible to fetch the record schema through it.")
+            .required(true)
+            .defaultValue(MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .allowableValues(MetadataStrategy.class)
+            .build();
+
+    static final PropertyDescriptor SCHEMA_READER = new PropertyDescriptor.Builder()
+            .name("schema-reader")
+            .displayName("Schema Reader")
+            .description("JsonTreeReader service to use for fetching the schema of records returned by the Airtable REST API")
+            .dependsOn(METADATA_STRATEGY, MetadataStrategy.USE_JSON_RECORD_READER.name())
+            .identifiesControllerService(JsonRecordReaderFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("record-writer")
+            .displayName("Record Writer")
+            .description("Service used for writing records returned by the Airtable REST API")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            BASE_ID,
+            TABLE_ID,
+            API_TOKEN,

Review Comment:
   Please move it up just after `API_URL`.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceeded;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor API_TOKEN = new PropertyDescriptor.Builder()
+            .name("api-token")
+            .displayName("API Token")
+            .description("The REST API token to use in queries. Should be generated on Airtable's account page.")

Review Comment:
   The Airtable documentation refers it as "API Key" so I would suggest using the same terminology.



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableRestService.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.airtable.service;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import javax.ws.rs.core.UriBuilder;
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.WebClientService;
+
+public class AirtableRestService {
+
+    public static final String API_V0_BASE_URL = "https://api.airtable.com/v0";
+
+    private final WebClientService webClientService;
+    private final String apiUrl;
+    private final String apiToken;
+    private final String baseId;
+    private final String tableId;
+
+    public AirtableRestService(final WebClientService webClientService,
+            final String apiUrl,
+            final String apiToken,
+            final String baseId,
+            final String tableId) {
+        this.webClientService = webClientService;
+        this.apiUrl = apiUrl;
+        this.apiToken = apiToken;
+        this.baseId = baseId;
+        this.tableId = tableId;
+    }
+
+    public byte[] getRecords(final AirtableGetRecordsParameters filter) throws RateLimitExceeded {
+        final URI uri = buildUri(filter);
+        try (final HttpResponseEntity response = webClientService.get()
+                .uri(uri)
+                .header("Authorization", "Bearer " + apiToken)
+                .retrieve()) {
+
+            if (response.statusCode() >= 200 && response.statusCode() < 300) {
+                return IOUtils.toByteArray(response.body());
+            }
+            if (response.statusCode() == 429) {

Review Comment:
   Please consider introducing constant for the status code 429 ("Too many requests") to increase readability.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/resources/docs/org.apache.nifi.processors.airtable.QueryAirtableTable/additionalDetails.html:
##########
@@ -0,0 +1,83 @@
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE html>
+<html lang="en" xmlns="http://www.w3.org/1999/html">
+<!--
+      Licensed to the Apache Software Foundation (ASF) under one or more

Review Comment:
   The "Licensed to the Apache Software Foundation" is duplicated by mistake.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/resources/docs/org.apache.nifi.processors.airtable.QueryAirtableTable/additionalDetails.html:
##########
@@ -0,0 +1,83 @@
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE html>
+<html lang="en" xmlns="http://www.w3.org/1999/html">
+<!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+
+<head>
+    <meta charset="utf-8"/>
+    <title>QueryAirtableTable</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+    <style>
+h2 {margin-top: 4em}
+h3 {margin-top: 3em}
+td {text-align: left}
+    </style>
+</head>
+
+<body>
+
+<h1>QueryAirtableTable</h1>
+
+<h3>Description</h3>
+<p>
+    Airtable is a spreadsheet-database hybrid. In Airtable an application is called base and each base can have multiple tables.
+    A table consists of records (rows) and each record can have multiple fields (columns).
+    The QueryAirtableTable processor can query records from a single base and table via Airtable's REST API.
+    The processor utilizes streams and NiFi record-based processing to be able to handle a large number of records and to allow arbitrary output format.
+    It can also split large record sets to multiple FlowFiles and transfer them in batches just like a database processor.
+</p>
+
+<h3>API Token</h3>
+<p>
+    Airtable REST API calls require a so-called "API Token" that can needs to be passed in a request. An Airtable account is required to generate such token.

Review Comment:
   typo, it should be  "needs to be passed" I guess



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/resources/docs/org.apache.nifi.processors.airtable.QueryAirtableTable/additionalDetails.html:
##########
@@ -0,0 +1,83 @@
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE html>
+<html lang="en" xmlns="http://www.w3.org/1999/html">
+<!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+
+<head>
+    <meta charset="utf-8"/>
+    <title>QueryAirtableTable</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+    <style>
+h2 {margin-top: 4em}
+h3 {margin-top: 3em}
+td {text-align: left}
+    </style>
+</head>
+
+<body>
+
+<h1>QueryAirtableTable</h1>
+
+<h3>Description</h3>
+<p>
+    Airtable is a spreadsheet-database hybrid. In Airtable an application is called base and each base can have multiple tables.
+    A table consists of records (rows) and each record can have multiple fields (columns).
+    The QueryAirtableTable processor can query records from a single base and table via Airtable's REST API.
+    The processor utilizes streams and NiFi record-based processing to be able to handle a large number of records and to allow arbitrary output format.
+    It can also split large record sets to multiple FlowFiles and transfer them in batches just like a database processor.
+</p>
+
+<h3>API Token</h3>
+<p>
+    Airtable REST API calls require a so-called "API Token" that can needs to be passed in a request. An Airtable account is required to generate such token.
+</p>
+
+<h3>API rate limit</h3>
+<p>
+    The Airtable REST API limits the number of requests that can be sent on a per-base basis to avoid bottlenecks. Currently, this limit is 5 requests per second per base.
+    If this limit is exceeded the user can't make another request for 30 seconds. It's the user's responsibility to handle this rate limit.
+</p>
+
+<h3>Metadata API</h3>
+<p>
+    Currently the Metadata API of Airtable is unstable, and we don't provide a way to use it. Until it becomes stable the user should set up a JsonTreeReader service to provide a schema.
+    The JsonTreeReader is need to be set up in a specific way to properly work with the response of an Airtable REST API call.

Review Comment:
   typo, "it needs to be"



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,107 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-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>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+        </dependency>

Review Comment:
   @dam4rus Please also remove unused `nifi-mock-record-utils` and `nifi-schema-registry-service-api` test dependencies as well.



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

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

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


[GitHub] [nifi] dam4rus commented on a diff in pull request #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.airtable.record.AirtableRecordSet;
+import org.apache.nifi.processors.airtable.record.AirtableJsonTreeRowRecordReaderFactory;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.AirtableRestService.RateLimitExceededException;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.JsonRecordReaderFactory;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.WriteResult;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.RecordSet;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer."),
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile.")
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table Name or ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")

Review Comment:
   Yes, it's stated in the API documentation. Sadly, I can't provide a link to the API documentation because Airtable generates API documentations for each base and table, which can be accessed only with an account that has access to that specific base. Here's the relevant part:
   
   ```
   pageSize
   number
   optional
   The number of records returned in each request. Must be less than or equal to 100. Default is 100. See the Pagination section below for more.
   ```



-- 
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 #6350: NIFI-10242 Create QueryAirtableTable processor

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


##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")

Review Comment:
   ```suggestion
           + " This processor is intended to be run on the Primary Node only.")
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/resources/docs/org.apache.nifi.processors.airtable.QueryAirtableTable/additionalDetails.html:
##########
@@ -0,0 +1,68 @@
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE html>
+<html lang="en" xmlns="http://www.w3.org/1999/html">
+<head>
+    <meta charset="utf-8"/>
+    <title>QueryAirtableTable</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+    <style>
+h2 {margin-top: 4em}
+h3 {margin-top: 3em}
+td {text-align: left}
+    </style>
+</head>
+
+<body>
+
+<h1>QueryAirtableTable</h1>

Review Comment:
   The Additional Details documentation contains a couple of references to record processing.
   Please remove/rephrase them.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "

Review Comment:
   'Max Records Per Flow File' property has been renamed to 'Max Records Per FlowFile'.
   Please adjust `@WritesAttribute` tags accordingly.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/pom.xml:
##########
@@ -0,0 +1,107 @@
+<?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-airtable-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-airtable-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>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+        </dependency>

Review Comment:
   These dependencies are not used anymore because record processing has been reverted.
   
   Also `nifi-mock-record-utils` and `nifi-schema-registry-service-api` test dependencies below.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per Flow File' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet. If Output Batch Size is set, then this "
+                + "attribute will not be populated."),

Review Comment:
   'Output Batch Size' is not used in this processor so the reference should be removed.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per Flow File' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet. If Output Batch Size is set, then this "
+                + "attribute will not be populated."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per Flow File' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order  "

Review Comment:
   ```suggestion
                   + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order "
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/parse/AirtableTableRetriever.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.airtable.parse;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator.Feature;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+
+public class AirtableTableRetriever {
+
+    static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory()
+            .configure(Feature.AUTO_CLOSE_JSON_CONTENT, false);
+
+    final AirtableRestService airtableRestService;
+    final AirtableGetRecordsParameters getRecordsParameters;
+    final Integer maxRecordsPerFlowFile;
+
+    public AirtableTableRetriever(final AirtableRestService airtableRestService,
+            final AirtableGetRecordsParameters getRecordsParameters,
+            final Integer maxRecordsPerFlowFile) {
+        this.airtableRestService = airtableRestService;
+        this.getRecordsParameters = getRecordsParameters;
+        this.maxRecordsPerFlowFile = maxRecordsPerFlowFile;
+    }
+
+    public AirtableRetrieveTableResult retrieveAll(final ProcessSession session) throws IOException {
+        int totalRecordCount = 0;
+        final List<FlowFile> flowFiles = new ArrayList<>();
+        AirtableRetrievePageResult retrievePageResult = null;
+        do {
+            retrievePageResult = retrieveNextPage(session, Optional.ofNullable(retrievePageResult));
+            totalRecordCount += retrievePageResult.getParsedRecordCount();
+            flowFiles.addAll(retrievePageResult.getFlowFiles());
+        } while (retrievePageResult.getNextOffset().isPresent());
+
+        retrievePageResult.getOngoingRecordSetFlowFileWriter()
+                .map(writer -> {
+                    try {
+                        return writer.closeRecordSet(session);
+                    } catch (IOException e) {
+                        throw new ProcessException("Failed to close Airtable record writer", e);
+                    }
+                })
+                .ifPresent(flowFiles::add);
+        return new AirtableRetrieveTableResult(flowFiles, totalRecordCount);
+    }
+
+    private AirtableRetrievePageResult retrieveNextPage(final ProcessSession session, final Optional<AirtableRetrievePageResult> previousPagePageResult)

Review Comment:
   Possible typo: `previousPageResult` ?



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per Flow File' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet. If Output Batch Size is set, then this "
+                + "attribute will not be populated."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per Flow File' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order  "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(QUERY_LAG_SECONDS)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final AirtableRetrieveTableResult retrieveTableResult;
+        try {
+            final AirtableTableRetriever tableRetriever = new AirtableTableRetriever(airtableRestService, getRecordsParameters, maxRecordsPerFlowFile);
+            retrieveTableResult = tableRetriever.retrieveAll(session);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to read Airtable records", e);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("Airtable REST API rate limit exceeded while reading records", e);
+        }
+
+        final List<FlowFile> flowFiles = retrieveTableResult.getFlowFiles();
+        if (flowFiles.isEmpty()) {
+            return;
+        }
+
+        if (maxRecordsPerFlowFile != null && maxRecordsPerFlowFile > 0) {
+            fragmentFlowFiles(session, flowFiles);
+        }
+        transferFlowFiles(session, flowFiles, retrieveTableResult.getTotalRecordCount());
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to update cluster state", e);
+        }
+    }
+
+    private AirtableGetRecordsParameters buildGetRecordsParameters(final ProcessContext context,
+            final String lastRecordFetchTime,
+            final String nowDateTimeString) {
+        final String fieldsProperty = context.getProperty(FIELDS).evaluateAttributeExpressions().getValue();
+        final String customFilter = context.getProperty(CUSTOM_FILTER).evaluateAttributeExpressions().getValue();
+        final Integer pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final AirtableGetRecordsParameters.Builder getRecordsParametersBuilder = new AirtableGetRecordsParameters.Builder();
+        if (lastRecordFetchTime != null) {
+            getRecordsParametersBuilder
+                    .modifiedAfter(lastRecordFetchTime)
+                    .modifiedBefore(nowDateTimeString);
+        }
+        if (fieldsProperty != null) {
+            getRecordsParametersBuilder.fields(Arrays.stream(fieldsProperty.split(",")).map(String::trim).collect(Collectors.toList()));
+        }
+        getRecordsParametersBuilder.customFilter(customFilter);
+        if (pageSize != null && pageSize > 0) {
+            getRecordsParametersBuilder.pageSize(pageSize);
+        }
+
+        return getRecordsParametersBuilder.build();
+    }
+
+    private void fragmentFlowFiles(final ProcessSession session, final List<FlowFile> flowFiles) {
+        final String fragmentIdentifier = UUID.randomUUID().toString();
+        for (int i = 0; i < flowFiles.size(); i++) {
+            final Map<String, String> fragmentAttributes = new HashMap<>();
+            fragmentAttributes.put(FRAGMENT_ID.key(), fragmentIdentifier);
+            fragmentAttributes.put(FRAGMENT_INDEX.key(), String.valueOf(i));
+            fragmentAttributes.put(FRAGMENT_COUNT.key(), String.valueOf(flowFiles.size()));
+
+            flowFiles.set(i, session.putAllAttributes(flowFiles.get(i), fragmentAttributes));
+        }
+    }
+
+    private void transferFlowFiles(final ProcessSession session, final List<FlowFile> flowFiles, final int totalRecordCount) {
+        session.transfer(flowFiles, REL_SUCCESS);
+        session.adjustCounter("Records Processed", totalRecordCount, false);
+        final String flowFilesAsString = flowFiles.stream().map(FlowFile::toString).collect(Collectors.joining(", ", "[", "]"));
+        getLogger().info("Transferred FlowFiles [{}] Records [{}]", flowFilesAsString, totalRecordCount);

Review Comment:
   I would use `debug` log here.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/service/AirtableRestService.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.airtable.service;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.Range;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+public class AirtableRestService {
+
+    public static final String API_V0_BASE_URL = "https://api.airtable.com/v0";
+
+    private static final int TOO_MANY_REQUESTS = 429;
+    private static final Range<Integer> SUCCESSFUL_RESPONSE_RANGE = Range.between(200, 299);
+
+    private final WebClientServiceProvider webClientServiceProvider;
+    private final String apiUrl;
+    private final String apiKey;
+    private final String baseId;
+    private final String tableId;
+
+    public AirtableRestService(final WebClientServiceProvider webClientServiceProvider,
+            final String apiUrl,
+            final String apiKey,
+            final String baseId,
+            final String tableId) {
+        this.webClientServiceProvider = webClientServiceProvider;
+        this.apiUrl = apiUrl;
+        this.apiKey = apiKey;
+        this.baseId = baseId;
+        this.tableId = tableId;
+    }
+
+    public InputStream getRecords(final AirtableGetRecordsParameters filter) throws RateLimitExceededException {
+        final URI uri = buildUri(filter);
+        final HttpResponseEntity response = webClientServiceProvider.getWebClientService()
+                .get()
+                .uri(uri)
+                .header("Authorization", "Bearer " + apiKey)
+                .retrieve();
+
+        try {
+            if (SUCCESSFUL_RESPONSE_RANGE.contains(response.statusCode())) {
+                return response.body();
+            }
+            if (response.statusCode() == TOO_MANY_REQUESTS) {
+                throw new RateLimitExceededException();
+            }
+            final StringBuilder exceptionMessageBuilder = new StringBuilder("Invalid response. Code: " + response.statusCode());

Review Comment:
   "Invalid response" is not the right message because when you get e.g. 401 due to wrong credentials, that is a valid response from the server. "Error response" or similar may be better.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/parse/AirtableRecordSetFlowFileWriter.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.airtable.parse;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import java.io.IOException;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.ProcessSession;
+
+public class AirtableRecordSetFlowFileWriter {
+
+    private final FlowFile flowFile;
+    private final JsonGenerator jsonGenerator;
+    private int recordCount = 0;
+
+    private AirtableRecordSetFlowFileWriter(final FlowFile flowFile, final JsonGenerator jsonGenerator) {
+        this.flowFile = flowFile;
+        this.jsonGenerator = jsonGenerator;
+    }
+
+    public static AirtableRecordSetFlowFileWriter startRecordSet(final ProcessSession session) throws
+            IOException {

Review Comment:
   ```suggestion
       public static AirtableRecordSetFlowFileWriter startRecordSet(final ProcessSession session) throws IOException {
   ```



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per Flow File' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet. If Output Batch Size is set, then this "
+                + "attribute will not be populated."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per Flow File' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order  "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(QUERY_LAG_SECONDS)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final AirtableRetrieveTableResult retrieveTableResult;
+        try {
+            final AirtableTableRetriever tableRetriever = new AirtableTableRetriever(airtableRestService, getRecordsParameters, maxRecordsPerFlowFile);
+            retrieveTableResult = tableRetriever.retrieveAll(session);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to read Airtable records", e);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("Airtable REST API rate limit exceeded while reading records", e);
+        }
+
+        final List<FlowFile> flowFiles = retrieveTableResult.getFlowFiles();
+        if (flowFiles.isEmpty()) {
+            return;
+        }
+
+        if (maxRecordsPerFlowFile != null && maxRecordsPerFlowFile > 0) {
+            fragmentFlowFiles(session, flowFiles);
+        }
+        transferFlowFiles(session, flowFiles, retrieveTableResult.getTotalRecordCount());
+
+        final Map<String, String> newState = new HashMap<>(state.toMap());
+        newState.put(LAST_RECORD_FETCH_TIME, currentRecordFetchDateTime);
+        try {
+            context.getStateManager().setState(newState, Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to update cluster state", e);
+        }
+    }
+
+    private AirtableGetRecordsParameters buildGetRecordsParameters(final ProcessContext context,
+            final String lastRecordFetchTime,
+            final String nowDateTimeString) {
+        final String fieldsProperty = context.getProperty(FIELDS).evaluateAttributeExpressions().getValue();
+        final String customFilter = context.getProperty(CUSTOM_FILTER).evaluateAttributeExpressions().getValue();
+        final Integer pageSize = context.getProperty(PAGE_SIZE).evaluateAttributeExpressions().asInteger();
+
+        final AirtableGetRecordsParameters.Builder getRecordsParametersBuilder = new AirtableGetRecordsParameters.Builder();
+        if (lastRecordFetchTime != null) {
+            getRecordsParametersBuilder
+                    .modifiedAfter(lastRecordFetchTime)
+                    .modifiedBefore(nowDateTimeString);
+        }
+        if (fieldsProperty != null) {
+            getRecordsParametersBuilder.fields(Arrays.stream(fieldsProperty.split(",")).map(String::trim).collect(Collectors.toList()));
+        }
+        getRecordsParametersBuilder.customFilter(customFilter);
+        if (pageSize != null && pageSize > 0) {
+            getRecordsParametersBuilder.pageSize(pageSize);
+        }
+
+        return getRecordsParametersBuilder.build();
+    }
+
+    private void fragmentFlowFiles(final ProcessSession session, final List<FlowFile> flowFiles) {
+        final String fragmentIdentifier = UUID.randomUUID().toString();
+        for (int i = 0; i < flowFiles.size(); i++) {
+            final Map<String, String> fragmentAttributes = new HashMap<>();
+            fragmentAttributes.put(FRAGMENT_ID.key(), fragmentIdentifier);
+            fragmentAttributes.put(FRAGMENT_INDEX.key(), String.valueOf(i));
+            fragmentAttributes.put(FRAGMENT_COUNT.key(), String.valueOf(flowFiles.size()));
+
+            flowFiles.set(i, session.putAllAttributes(flowFiles.get(i), fragmentAttributes));
+        }
+    }
+
+    private void transferFlowFiles(final ProcessSession session, final List<FlowFile> flowFiles, final int totalRecordCount) {
+        session.transfer(flowFiles, REL_SUCCESS);

Review Comment:
   The processor should generate a `RECEIVE` provenance event for each successful FlowFile.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/QueryAirtableTable.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.airtable;
+
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_COUNT;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_ID;
+import static org.apache.nifi.flowfile.attributes.FragmentAttributes.FRAGMENT_INDEX;
+import static org.apache.nifi.processors.airtable.service.AirtableRestService.API_V0_BASE_URL;
+
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+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.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.configuration.DefaultSettings;
+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.util.StandardValidators;
+import org.apache.nifi.processors.airtable.parse.AirtableRetrieveTableResult;
+import org.apache.nifi.processors.airtable.parse.AirtableTableRetriever;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+import org.apache.nifi.processors.airtable.service.RateLimitExceededException;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@PrimaryNodeOnly
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@TriggerWhenEmpty
+@Tags({"airtable", "query", "database"})
+@CapabilityDescription("Query records from an Airtable table. Records are incrementally retrieved based on the last modified time of the records."
+        + " Records can also be further filtered by setting the 'Custom Filter' property which supports the formulas provided by the Airtable API."
+        + " Schema can be provided by setting up a JsonTreeReader controller service properly. This processor is intended to be run on the Primary Node only.")
+@Stateful(scopes = Scope.CLUSTER, description = "The last successful query's time is stored in order to enable incremental loading."
+        + " The initial query returns all the records in the table and each subsequent query filters the records by their last modified time."
+        + " In other words, if a record is updated after the last successful query only the updated records will be returned in the next query."
+        + " 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.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.count", description = "Sets the number of records in the FlowFile."),
+        @WritesAttribute(attribute = "fragment.identifier", description = "If 'Max Records Per Flow File' is set then all FlowFiles from the same query result set "
+                + "will have the same value for the fragment.identifier attribute. This can then be used to correlate the results."),
+        @WritesAttribute(attribute = "fragment.count", description = "If 'Max Records Per Flow File' is set then this is the total number of "
+                + "FlowFiles produced by a single ResultSet. This can be used in conjunction with the "
+                + "fragment.identifier attribute in order to know how many FlowFiles belonged to the same incoming ResultSet. If Output Batch Size is set, then this "
+                + "attribute will not be populated."),
+        @WritesAttribute(attribute = "fragment.index", description = "If 'Max Records Per Flow File' is set then the position of this FlowFile in the list of "
+                + "outgoing FlowFiles that were all derived from the same result set FlowFile. This can be "
+                + "used in conjunction with the fragment.identifier attribute to know which FlowFiles originated from the same query result set and in what order  "
+                + "FlowFiles were produced"),
+})
+@DefaultSettings(yieldDuration = "35 sec")
+public class QueryAirtableTable extends AbstractProcessor {
+
+    static final PropertyDescriptor API_URL = new PropertyDescriptor.Builder()
+            .name("api-url")
+            .displayName("API URL")
+            .description("The URL for the Airtable REST API including the domain and the path to the API (e.g. https://api.airtable.com/v0).")
+            .defaultValue(API_V0_BASE_URL)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
+            .name("api-key")
+            .displayName("API Key")
+            .description("The REST API key to use in queries. Should be generated on Airtable's account page.")
+            .required(true)
+            .sensitive(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor BASE_ID = new PropertyDescriptor.Builder()
+            .name("base-id")
+            .displayName("Base ID")
+            .description("The ID of the Airtable base to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_ID = new PropertyDescriptor.Builder()
+            .name("table-id")
+            .displayName("Table ID")
+            .description("The name or the ID of the Airtable table to be queried.")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
+            .name("fields")
+            .displayName("Fields")
+            .description("Comma-separated list of fields to query from the table. Both the field's name and ID can be used.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor CUSTOM_FILTER = new PropertyDescriptor.Builder()
+            .name("custom-filter")
+            .displayName("Custom Filter")
+            .description("Filter records by Airtable's formulas.")
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
+            .name("page-size")
+            .displayName("Page Size")
+            .description("Number of records to be fetched in a page. Should be between 0 and 100 inclusively.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createLongValidator(0, 100, true))
+            .build();
+
+    static final PropertyDescriptor MAX_RECORDS_PER_FLOW_FILE = new PropertyDescriptor.Builder()
+            .name("max-records-per-flow-file")
+            .displayName("Max Records Per FlowFile")
+            .description("The maximum number of result records that will be included in a single FlowFile. This will allow you to break up very large"
+                    + " result sets into multiple FlowFiles. If the value specified is zero, then all records are returned in a single FlowFile.")
+            .defaultValue("0")
+            .required(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor WEB_CLIENT_SERVICE_PROVIDER = new PropertyDescriptor.Builder()
+            .name("web-client-service-provider")
+            .displayName("Web Client Service Provider")
+            .description("Web Client Service Provider to use for Airtable REST API requests")
+            .identifiesControllerService(WebClientServiceProvider.class)
+            .required(true)
+            .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 List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            API_URL,
+            API_KEY,
+            BASE_ID,
+            TABLE_ID,
+            FIELDS,
+            CUSTOM_FILTER,
+            PAGE_SIZE,
+            MAX_RECORDS_PER_FLOW_FILE,
+            WEB_CLIENT_SERVICE_PROVIDER
+    ));
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+
+    private static final String LAST_RECORD_FETCH_TIME = "last_record_fetch_time";
+    private static final int QUERY_LAG_SECONDS = 1;
+
+    private volatile AirtableRestService airtableRestService;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return RELATIONSHIPS;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final String apiUrl = context.getProperty(API_URL).evaluateAttributeExpressions().getValue();
+        final String apiKey = context.getProperty(API_KEY).getValue();
+        final String baseId = context.getProperty(BASE_ID).evaluateAttributeExpressions().getValue();
+        final String tableId = context.getProperty(TABLE_ID).evaluateAttributeExpressions().getValue();
+        final WebClientServiceProvider webClientServiceProvider = context.getProperty(WEB_CLIENT_SERVICE_PROVIDER).asControllerService(WebClientServiceProvider.class);
+        airtableRestService = new AirtableRestService(webClientServiceProvider, apiUrl, apiKey, baseId, tableId);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final Integer maxRecordsPerFlowFile = context.getProperty(MAX_RECORDS_PER_FLOW_FILE).evaluateAttributeExpressions().asInteger();
+
+        final StateMap state;
+        try {
+            state = context.getStateManager().getState(Scope.CLUSTER);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to get cluster state", e);
+        }
+
+        final String lastRecordFetchDateTime = state.get(LAST_RECORD_FETCH_TIME);
+        final String currentRecordFetchDateTime = OffsetDateTime.now()
+                .minusSeconds(QUERY_LAG_SECONDS)
+                .format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
+
+        final AirtableGetRecordsParameters getRecordsParameters = buildGetRecordsParameters(context, lastRecordFetchDateTime, currentRecordFetchDateTime);
+        final AirtableRetrieveTableResult retrieveTableResult;
+        try {
+            final AirtableTableRetriever tableRetriever = new AirtableTableRetriever(airtableRestService, getRecordsParameters, maxRecordsPerFlowFile);
+            retrieveTableResult = tableRetriever.retrieveAll(session);
+        } catch (IOException e) {
+            throw new ProcessException("Failed to read Airtable records", e);
+        } catch (RateLimitExceededException e) {
+            context.yield();
+            throw new ProcessException("Airtable REST API rate limit exceeded while reading records", e);
+        }
+
+        final List<FlowFile> flowFiles = retrieveTableResult.getFlowFiles();
+        if (flowFiles.isEmpty()) {
+            return;

Review Comment:
   The processor should yield when no new records available. Otherwise it will ping the Airtable service in a quick loop.



##########
nifi-nar-bundles/nifi-airtable-bundle/nifi-airtable-processors/src/main/java/org/apache/nifi/processors/airtable/parse/AirtableTableRetriever.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.airtable.parse;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator.Feature;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.airtable.service.AirtableGetRecordsParameters;
+import org.apache.nifi.processors.airtable.service.AirtableRestService;
+
+public class AirtableTableRetriever {
+
+    static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    static final JsonFactory JSON_FACTORY = OBJECT_MAPPER.getFactory()
+            .configure(Feature.AUTO_CLOSE_JSON_CONTENT, false);
+
+    final AirtableRestService airtableRestService;
+    final AirtableGetRecordsParameters getRecordsParameters;
+    final Integer maxRecordsPerFlowFile;
+
+    public AirtableTableRetriever(final AirtableRestService airtableRestService,
+            final AirtableGetRecordsParameters getRecordsParameters,
+            final Integer maxRecordsPerFlowFile) {
+        this.airtableRestService = airtableRestService;
+        this.getRecordsParameters = getRecordsParameters;
+        this.maxRecordsPerFlowFile = maxRecordsPerFlowFile;
+    }
+
+    public AirtableRetrieveTableResult retrieveAll(final ProcessSession session) throws IOException {
+        int totalRecordCount = 0;
+        final List<FlowFile> flowFiles = new ArrayList<>();
+        AirtableRetrievePageResult retrievePageResult = null;
+        do {
+            retrievePageResult = retrieveNextPage(session, Optional.ofNullable(retrievePageResult));
+            totalRecordCount += retrievePageResult.getParsedRecordCount();
+            flowFiles.addAll(retrievePageResult.getFlowFiles());
+        } while (retrievePageResult.getNextOffset().isPresent());
+
+        retrievePageResult.getOngoingRecordSetFlowFileWriter()
+                .map(writer -> {
+                    try {
+                        return writer.closeRecordSet(session);
+                    } catch (IOException e) {
+                        throw new ProcessException("Failed to close Airtable record writer", e);
+                    }
+                })
+                .ifPresent(flowFiles::add);
+        return new AirtableRetrieveTableResult(flowFiles, totalRecordCount);
+    }
+
+    private AirtableRetrievePageResult retrieveNextPage(final ProcessSession session, final Optional<AirtableRetrievePageResult> previousPagePageResult)
+            throws IOException {
+        final AirtableGetRecordsParameters parameters = previousPagePageResult.flatMap(AirtableRetrievePageResult::getNextOffset)
+                .map(getRecordsParameters::withOffset)
+                .orElse(getRecordsParameters);
+        final InputStream inputStream = airtableRestService.getRecords(parameters);

Review Comment:
   This InputStream should be closed.
   It comes from `HttpResponseEntity` which is `AutoClosable` and it would close the stream if it was used in a TWR block. I see it would be complicated to restructure the code to process `HttpResponseEntity` in a single block. In this case we must close the InputStream ourselves.



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