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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
