You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2022/01/06 11:53:21 UTC

[GitHub] [drill] jnturton commented on a change in pull request #2414: DRILL-8092: Add Auto Pagination to HTTP Storage Plugin

jnturton commented on a change in pull request #2414:
URL: https://github.com/apache/drill/pull/2414#discussion_r776665346



##########
File path: contrib/storage-http/README.md
##########
@@ -4,7 +4,7 @@ The HTTP storage plugin lets you query APIs over HTTP/REST. The plugin
 expects JSON responses.
 
 The HTTP plugin is new in Drill 1.18 and is an Alpha feature. It works well, and we

Review comment:
       Should we drop this disclaimer now?  Drill 1.18 was a long time ago...

##########
File path: contrib/storage-http/README.md
##########
@@ -278,6 +278,21 @@ If the `authType` is set to `basic`, `username` and `password` must be set in th
 
 `password`: The password for basic authentication.
 
+#### Limiting Results
+Some APIs contain a field which is used to limit the number of results returned by the API.  If such a field is known, you can set the `limitField` parameter to this field name

Review comment:
       ```suggestion
   Some APIs support a query parameter which is used to limit the number of results returned by the API.  In this case you can set the `limitQueryParm` config variable to the query parameter name
   ```

##########
File path: contrib/storage-http/README.md
##########
@@ -278,6 +278,21 @@ If the `authType` is set to `basic`, `username` and `password` must be set in th
 
 `password`: The password for basic authentication.
 
+#### Limiting Results
+Some APIs contain a field which is used to limit the number of results returned by the API.  If such a field is known, you can set the `limitField` parameter to this field name
+ and Drill will automatically include this in your query.  For instance, if you have an API which has a limit field called `maxRecords`, if you set this option, and execute the
+  following query:
+  
+```sql
+SELECT <fields>
+FROM api.limitedApi
+LIMIT 10 
+```  
+Drill will send the following query to your API:

Review comment:
       ```suggestion
   Drill will send the following request to your API:
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/HttpBatchReader.java
##########
@@ -216,13 +246,15 @@ protected HttpProxyConfig proxySettings(Config drillConfig, HttpUrl url) {
 
   @Override
   public boolean next() {
-    recordCount++;
+    boolean result = jsonLoader.readBatch();
 
-    // Stop after the limit has been reached
-    if (maxRecords >= 1 && recordCount > maxRecords) {
-      return false;
+    // Allows limitless pagination.
+    if (paginator != null &&
+      maxRecords < 0 && (resultSetLoader.totalRowCount()) < paginator.getMaxPageSize()) {
+      logger.debug("Ending pagination: Pages too small");

Review comment:
       ```suggestion
         logger.debug("Partially filled page received, ending pagination");
   ```

##########
File path: contrib/storage-http/README.md
##########
@@ -278,6 +278,21 @@ If the `authType` is set to `basic`, `username` and `password` must be set in th
 
 `password`: The password for basic authentication.
 
+#### Limiting Results
+Some APIs contain a field which is used to limit the number of results returned by the API.  If such a field is known, you can set the `limitField` parameter to this field name
+ and Drill will automatically include this in your query.  For instance, if you have an API which has a limit field called `maxRecords`, if you set this option, and execute the

Review comment:
       ```suggestion
    and Drill will automatically include this in your query.  For instance, if you have an API which supports a limit query parameter called `maxRecords` and you set the abovementioned config variable then execute the
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/OffsetPaginator.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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class OffsetPaginator extends Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(OffsetPaginator.class);
+
+  private final int limit;
+  private final String limitField;
+  private final String offsetField;
+  private int offset;
+
+  /**
+   * This class implements the idea of an Offset Paginator. See here for complete explanation:
+   * https://nordicapis.com/everything-you-need-to-know-about-api-pagination/
+   * <p>
+   *
+   * @param builder     The okhttp3 URL builder which has the API root URL
+   * @param limit       The limit clause from the sql query
+   * @param maxPageSize The maximum page size the API documentation
+   * @param limitField  The field name which corresponds to the limit field from the API
+   * @param offsetField The field name which corresponds to the offset field from the API
+   */
+  public OffsetPaginator(Builder builder, int limit, int maxPageSize, String limitField, String offsetField) {
+    super(builder, paginationMode.OFFSET, maxPageSize, limit > 0);
+    this.limit = limit;
+    this.limitField = limitField;
+    this.offsetField = offsetField;
+    this.paginatedUrls = buildPaginatedURLs();
+    this.offset = 0;
+
+    // Page size must be greater than zero
+    if (maxPageSize <= 0) {
+      throw UserException
+              .validationError()
+              .message("API limit cannot be zero")
+              .build(logger);
+    }
+  }
+
+  public int getLimit() {
+    return limit;
+  }
+
+  @Override
+  public String next() {
+    if (hasLimit) {
+      return super.next();
+    } else {
+      return generateNextUrl();
+    }
+  }
+
+  @Override
+  public String generateNextUrl() {
+    builder.removeAllEncodedQueryParameters(offsetField);
+    builder.removeAllEncodedQueryParameters(limitField);
+
+    builder.addQueryParameter(offsetField, String.valueOf(offset));
+    builder.addQueryParameter(limitField, String.valueOf(maxPageSize));
+    offset += maxPageSize;
+
+    return builder.build().url().toString();
+  }
+
+
+  /**
+   * Build the paginated URLs.  If the parameters are invalid, return a list with the original URL.
+   *
+   * @return List of paginated URLs
+   */
+  @Override
+  public List<HttpUrl> buildPaginatedURLs() {
+    this.paginatedUrls = new ArrayList<>();
+    // If user wants 1000 records, and the max page size is 100, we need to send 10 requests

Review comment:
       ```suggestion
       // If user wants 1000 records, and the page size is 100, we need to send 10 requests
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/Paginator.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * This class is the abstraction for the Paginator class.  There are
+ * different pagination methods, however, all essentially require the query
+ * engine to generate URLs to retrieve the next batch of data and also
+ * to determine whether the URL has more data.
+ *
+ * The Offset and Page paginators work either with a limit or without, but function
+ * slightly differently.  If a limit is specified and pushed down, the paginator will
+ * generate a list of URLs with the appropriate pagination parameters.  In the future
+ * this could be parallelized, however in the V1 all these requests are run in series.
+ */
+public abstract class Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(Paginator.class);
+  private static final int MAX_ATTEMPTS = 100;
+  protected final int maxPageSize;
+  private boolean hasMore;
+
+  public enum paginationMode {
+    OFFSET,
+    PAGE
+  }
+
+  protected final boolean hasLimit;
+  protected final paginationMode MODE;
+  protected int index;
+  protected Builder builder;
+  protected List<HttpUrl> paginatedUrls;
+
+
+  public Paginator(Builder builder, paginationMode mode, int maxPageSize, boolean hasLimit) {
+    this.MODE = mode;
+    this.builder = builder;
+    this.maxPageSize = maxPageSize;
+    this.hasLimit = hasLimit;
+    hasMore = true;
+    index = 0;
+  }
+
+  public void setBuilder(Builder builder) {
+    this.builder = builder;
+  }
+
+  public abstract List<HttpUrl> buildPaginatedURLs();
+
+  /**
+   * This method is to be used when the user does not include a limit in the query
+   * In each paginator, the paginator tracks whether there is more data.  If there is
+   * more data, the paginator marks the hasMore variable false.
+   * @return
+   */
+  public abstract String generateNextUrl();
+
+  public List<HttpUrl> getPaginatedUrls() {
+    return this.paginatedUrls;
+  }
+
+  /**
+   * This method is used in pagination queries when no limit is present.  The intended
+   * flow is that if no limit is present, if the batch reader encounters a page which has
+   * less data than the max page size, the batch reader should call this method to stop

Review comment:
       ```suggestion
      * less data than the page size, the batch reader should call this method to stop
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/HttpApiConfig.java
##########
@@ -101,6 +100,8 @@
   @JsonProperty
   private final int xmlDataLevel;
   @JsonProperty
+  private final String limitField;

Review comment:
       (Etc. I won't comment on any more occurrences of this.)

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/OffsetPaginator.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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class OffsetPaginator extends Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(OffsetPaginator.class);
+
+  private final int limit;
+  private final String limitField;
+  private final String offsetField;
+  private int offset;
+
+  /**
+   * This class implements the idea of an Offset Paginator. See here for complete explanation:
+   * https://nordicapis.com/everything-you-need-to-know-about-api-pagination/
+   * <p>
+   *
+   * @param builder     The okhttp3 URL builder which has the API root URL
+   * @param limit       The limit clause from the sql query
+   * @param maxPageSize The maximum page size the API documentation

Review comment:
       ```suggestion
      * @param pageSize The number of results in a single page returned by the API
   ```
   To me this is just a "page size", rather than a "max page size".  The final page of results from a call need not be full, but that doesn't change its fixed size.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/PagePaginator.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class PagePaginator extends Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(OffsetPaginator.class);
+
+  private final int limit;
+  private final String pageField;
+  private final String pageSizeField;
+  private int currentPage;
+
+
+  /**
+   * The Page Paginator works similarly to the offset paginator.  It requires the user to supply a page number,
+   * a page size variable, and a maximum page size.
+   * @param builder
+   * @param limit
+   * @param maxPageSize
+   * @param pageField
+   * @param pageSizeField
+   */
+  public PagePaginator(Builder builder, int limit, int maxPageSize, String pageField, String pageSizeField) {

Review comment:
       ```suggestion
     public PagePaginator(Builder builder, int limit, int pageSize, String pageNumQueryParm, String pageSizeQueryParm) {
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/OffsetPaginator.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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class OffsetPaginator extends Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(OffsetPaginator.class);
+
+  private final int limit;
+  private final String limitField;
+  private final String offsetField;
+  private int offset;
+
+  /**
+   * This class implements the idea of an Offset Paginator. See here for complete explanation:
+   * https://nordicapis.com/everything-you-need-to-know-about-api-pagination/
+   * <p>
+   *
+   * @param builder     The okhttp3 URL builder which has the API root URL
+   * @param limit       The limit clause from the sql query
+   * @param maxPageSize The maximum page size the API documentation
+   * @param limitField  The field name which corresponds to the limit field from the API
+   * @param offsetField The field name which corresponds to the offset field from the API
+   */
+  public OffsetPaginator(Builder builder, int limit, int maxPageSize, String limitField, String offsetField) {
+    super(builder, paginationMode.OFFSET, maxPageSize, limit > 0);
+    this.limit = limit;
+    this.limitField = limitField;
+    this.offsetField = offsetField;
+    this.paginatedUrls = buildPaginatedURLs();
+    this.offset = 0;
+
+    // Page size must be greater than zero
+    if (maxPageSize <= 0) {
+      throw UserException
+              .validationError()
+              .message("API limit cannot be zero")

Review comment:
       ```suggestion
                 .message("API page size must be positive")
   ```

##########
File path: contrib/storage-http/Pagination.md
##########
@@ -0,0 +1,38 @@
+# Auto Pagination in Drill
+Remote APIs frequently implement some sort of pagination as a way of limiting results.  However, if you are performing bulk data analysis, it is necessary to reassemble the 
+data into one larger dataset.  Drill's auto-pagination features allow this to happen in the background, so that the user will get clean data back.
+
+To use a paginator, you simply have to configure the paginator in the connection for the particular API.  
+
+## Offset Pagination
+Offset Pagination uses commands similar to SQL which has a `LIMIT` and an `OFFSET`.  With an offset paginator, let's say you want 200 records and the maximum page size is 50 
+records, the offset paginator will break up your query into 4 requests as shown below:
+
+* myapi.com?limit=50&offset=0
+* myapi.com?limit=50?offset=50
+* myapi.com?limit=50&offset=100
+* myapi.com?limit=50&offset=150
+
+### Configuring Offset Pagination
+To configure an offset paginator, simply add the following to the configuration for your connection. 
+
+```json
+"paginator": {
+   "limitField": "<limit>",

Review comment:
       I found it confusing that the limit/offset/page config vars are called fooField, while they refer to query parameters.  For me, fooParm or even fooQueryParm would be a lot clearer because we commonly use the word field to refer to a column of data.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/HttpApiConfig.java
##########
@@ -101,6 +100,8 @@
   @JsonProperty
   private final int xmlDataLevel;
   @JsonProperty
+  private final String limitField;

Review comment:
       ```suggestion
     private final String limitQueryParm;
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/HttpBatchReader.java
##########
@@ -72,6 +88,7 @@ public boolean open(SchemaNegotiator negotiator) {
         .getString(ExecConstants.DRILL_TMP_DIR);
 
     HttpUrl url = buildUrl();
+    logger.debug("Final URL: {}", url.toString());

Review comment:
       (Micro optimisation) Can we drop the toString() here?  It will be invoked needlessly when the logging threshold is set higher than DEBUG.

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/Paginator.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * This class is the abstraction for the Paginator class.  There are
+ * different pagination methods, however, all essentially require the query
+ * engine to generate URLs to retrieve the next batch of data and also
+ * to determine whether the URL has more data.
+ *
+ * The Offset and Page paginators work either with a limit or without, but function
+ * slightly differently.  If a limit is specified and pushed down, the paginator will
+ * generate a list of URLs with the appropriate pagination parameters.  In the future
+ * this could be parallelized, however in the V1 all these requests are run in series.
+ */
+public abstract class Paginator {

Review comment:
       Did you consider `class Paginator implements java.lang.Iterable`?  Looks like it might fit...

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/Paginator.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * This class is the abstraction for the Paginator class.  There are
+ * different pagination methods, however, all essentially require the query
+ * engine to generate URLs to retrieve the next batch of data and also
+ * to determine whether the URL has more data.
+ *
+ * The Offset and Page paginators work either with a limit or without, but function
+ * slightly differently.  If a limit is specified and pushed down, the paginator will
+ * generate a list of URLs with the appropriate pagination parameters.  In the future
+ * this could be parallelized, however in the V1 all these requests are run in series.
+ */
+public abstract class Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(Paginator.class);
+  private static final int MAX_ATTEMPTS = 100;
+  protected final int maxPageSize;

Review comment:
       ```suggestion
     protected final int pageSize;
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/PagePaginator.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class PagePaginator extends Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(OffsetPaginator.class);
+
+  private final int limit;
+  private final String pageField;
+  private final String pageSizeField;
+  private int currentPage;
+
+
+  /**
+   * The Page Paginator works similarly to the offset paginator.  It requires the user to supply a page number,
+   * a page size variable, and a maximum page size.

Review comment:
       ```suggestion
      * a page number query parameter, and a page size.
   ```

##########
File path: contrib/storage-http/src/main/java/org/apache/drill/exec/store/http/paginator/OffsetPaginator.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.drill.exec.store.http.paginator;
+
+import okhttp3.HttpUrl;
+import okhttp3.HttpUrl.Builder;
+import org.apache.drill.common.exceptions.UserException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class OffsetPaginator extends Paginator {
+
+  private static final Logger logger = LoggerFactory.getLogger(OffsetPaginator.class);
+
+  private final int limit;
+  private final String limitField;
+  private final String offsetField;
+  private int offset;
+
+  /**
+   * This class implements the idea of an Offset Paginator. See here for complete explanation:
+   * https://nordicapis.com/everything-you-need-to-know-about-api-pagination/
+   * <p>
+   *
+   * @param builder     The okhttp3 URL builder which has the API root URL
+   * @param limit       The limit clause from the sql query
+   * @param maxPageSize The maximum page size the API documentation
+   * @param limitField  The field name which corresponds to the limit field from the API
+   * @param offsetField The field name which corresponds to the offset field from the API
+   */
+  public OffsetPaginator(Builder builder, int limit, int maxPageSize, String limitField, String offsetField) {
+    super(builder, paginationMode.OFFSET, maxPageSize, limit > 0);
+    this.limit = limit;
+    this.limitField = limitField;
+    this.offsetField = offsetField;
+    this.paginatedUrls = buildPaginatedURLs();
+    this.offset = 0;
+
+    // Page size must be greater than zero
+    if (maxPageSize <= 0) {
+      throw UserException
+              .validationError()
+              .message("API limit cannot be zero")
+              .build(logger);
+    }
+  }
+
+  public int getLimit() {
+    return limit;
+  }
+
+  @Override
+  public String next() {
+    if (hasLimit) {
+      return super.next();
+    } else {
+      return generateNextUrl();
+    }
+  }
+
+  @Override
+  public String generateNextUrl() {
+    builder.removeAllEncodedQueryParameters(offsetField);
+    builder.removeAllEncodedQueryParameters(limitField);
+
+    builder.addQueryParameter(offsetField, String.valueOf(offset));
+    builder.addQueryParameter(limitField, String.valueOf(maxPageSize));
+    offset += maxPageSize;
+
+    return builder.build().url().toString();
+  }
+
+
+  /**
+   * Build the paginated URLs.  If the parameters are invalid, return a list with the original URL.
+   *
+   * @return List of paginated URLs
+   */
+  @Override
+  public List<HttpUrl> buildPaginatedURLs() {
+    this.paginatedUrls = new ArrayList<>();

Review comment:
       I think this should be a method-local variable, not the object member variable which gets assigned to the result of this method




-- 
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: dev-unsubscribe@drill.apache.org

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