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/07/13 13:13:05 UTC

[GitHub] [drill] jnturton commented on a diff in pull request #2585: DRILL-8235: Add Storage Plugin for Google Sheets

jnturton commented on code in PR #2585:
URL: https://github.com/apache/drill/pull/2585#discussion_r919821083


##########
contrib/native/client/patches/zookeeper-3.4.6-x64.patch:
##########
@@ -1,163 +0,0 @@
-From 64697ddd8a90f29d1693658f04e975e435e3c869 Mon Sep 17 00:00:00 2001

Review Comment:
   An accidental deletion? I have to admit that I'm not sure what the history of this patch is or why we carry it...



##########
logical/src/main/java/org/apache/drill/common/logical/StoragePluginConfig.java:
##########
@@ -33,6 +34,7 @@
 
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
 @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+@JsonFormat(with = JsonFormat.Feature.ACCEPT_CASE_INSENSITIVE_PROPERTIES)

Review Comment:
   I think I'm probably in favour of case insensitive config properties but only if we can do it consistently. So have we also got case insensitivity for format config properties? Does that carry through to the table function format config override syntax?



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 
+  `http://localhost:8047/credentials/<plugin name>/update_oauth2_authtoken`.
+
+1. To obtain the `clientID` and `client_secret` you will need to obtain the Google keys, open the [Google Sheets API](https://console.cloud.google.com/apis/library/sheets.googleapis.com) and click on the `Enable` button. 
+2. Once you've enabled teh API, you will be taken to the API Manager.  Either select a pre-existing project or create a new one.
+3. Next, navigate to the `Credentials` in the left panel.
+4. Click on `+Create Credentials` at the top of the page.  Select `OAuth client ID` and select `Web Application` or `Desktop` as the type.  Follow the instructions and download 
+   the JSON file that Google provides.
+
+Drill does not use the JSON file, but you will be cutting and pasting values from the JSON file into the Drill configuration.
+
+## Setup Step 2:  Configure Drill
+Create a storage plugin following the normal procedure for doing so.  You can use the example below as a template.  Cut and paste the `clientID` and `client_secret` from the 
+JSON file into your Drill configuration as shown below.  Once you've done that, save the configuration.
+
+```json
+{
+  "type": "googlesheets",
+  "allTextMode": true,
+  "extractHeaders": true,
+  "oAuthConfig": {
+    "callbackURL": "http://localhost:8047/credentials/googlesheets/update_oauth2_authtoken",
+    "authorizationURL": "https://accounts.google.com/o/oauth2/auth",
+    "authorizationParams": {
+      "response_type": "code",
+      "scope": "https://www.googleapis.com/auth/spreadsheets"
+    }
+  },
+  "credentialsProvider": {
+    "credentialsProviderType": "PlainCredentialsProvider",
+    "credentials": {
+      "clientID": "<YOUR CLIENT ID>",
+      "clientSecret": "<YOUR CLIENT SECRET>",
+      "tokenURI": "https://oauth2.googleapis.com/token"
+    },
+    "userCredentials": {}
+  },
+  "enabled": true,
+  "authMode": "SHARED_USER"
+}
+```
+
+With the exception of the clientID, client_secret and redirects, you should not have to modify any of the other parameters in the configuration. 
+
+### Other Configuration Parameters
+
+There are two configuration parameters which you may want to adjust:
+* `allTextMode`:  This parameter when `true` disables Drill's data type inferencing for your files.  If your data has inconsistent data types, set this to `true`.  Default is 
+  `true`. 
+* `extractHeaders`:  When `true`, Drill will treat the first row of your data as headers.  When `false` Drill will assign column names like `field_n` for each column.
+
+### Authenticating with Google
+Once you have configured Drill to query GoogleSheets, there is one final step before you can access data.  You must authenticate the application (Drill) with GoogleSheets.  After you have saved your GoogleSheets configuration, navigate back to the configuration screen for your plugin and click on `Authorize`. A new window should appear which will prompt you to authenticate with Google services.  Once you have done that, you should be able to query GoogleSheets!  See, that wasn't so hard!

Review Comment:
   ```suggestion
   Once you have configured Drill to query Google Sheets, there is one final step before you can access data.  You must authenticate the application (Drill) with Google Sheets.  After you have saved your Google Sheets configuration, navigate back to the configuration screen for your plugin and click on `Authorize`. A new window should appear which will prompt you to authenticate with Google services.  Once you have done that, you should be able to query Google Sheets!  See, that wasn't so hard!
   ```



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 
+  `http://localhost:8047/credentials/<plugin name>/update_oauth2_authtoken`.
+
+1. To obtain the `clientID` and `client_secret` you will need to obtain the Google keys, open the [Google Sheets API](https://console.cloud.google.com/apis/library/sheets.googleapis.com) and click on the `Enable` button. 
+2. Once you've enabled teh API, you will be taken to the API Manager.  Either select a pre-existing project or create a new one.
+3. Next, navigate to the `Credentials` in the left panel.
+4. Click on `+Create Credentials` at the top of the page.  Select `OAuth client ID` and select `Web Application` or `Desktop` as the type.  Follow the instructions and download 
+   the JSON file that Google provides.
+
+Drill does not use the JSON file, but you will be cutting and pasting values from the JSON file into the Drill configuration.
+
+## Setup Step 2:  Configure Drill
+Create a storage plugin following the normal procedure for doing so.  You can use the example below as a template.  Cut and paste the `clientID` and `client_secret` from the 
+JSON file into your Drill configuration as shown below.  Once you've done that, save the configuration.
+
+```json
+{
+  "type": "googlesheets",
+  "allTextMode": true,
+  "extractHeaders": true,
+  "oAuthConfig": {
+    "callbackURL": "http://localhost:8047/credentials/googlesheets/update_oauth2_authtoken",
+    "authorizationURL": "https://accounts.google.com/o/oauth2/auth",
+    "authorizationParams": {
+      "response_type": "code",
+      "scope": "https://www.googleapis.com/auth/spreadsheets"
+    }
+  },
+  "credentialsProvider": {
+    "credentialsProviderType": "PlainCredentialsProvider",
+    "credentials": {
+      "clientID": "<YOUR CLIENT ID>",
+      "clientSecret": "<YOUR CLIENT SECRET>",
+      "tokenURI": "https://oauth2.googleapis.com/token"
+    },
+    "userCredentials": {}
+  },
+  "enabled": true,
+  "authMode": "SHARED_USER"
+}
+```
+
+With the exception of the clientID, client_secret and redirects, you should not have to modify any of the other parameters in the configuration. 
+
+### Other Configuration Parameters
+
+There are two configuration parameters which you may want to adjust:
+* `allTextMode`:  This parameter when `true` disables Drill's data type inferencing for your files.  If your data has inconsistent data types, set this to `true`.  Default is 
+  `true`. 
+* `extractHeaders`:  When `true`, Drill will treat the first row of your data as headers.  When `false` Drill will assign column names like `field_n` for each column.
+
+### Authenticating with Google
+Once you have configured Drill to query GoogleSheets, there is one final step before you can access data.  You must authenticate the application (Drill) with GoogleSheets.  After you have saved your GoogleSheets configuration, navigate back to the configuration screen for your plugin and click on `Authorize`. A new window should appear which will prompt you to authenticate with Google services.  Once you have done that, you should be able to query GoogleSheets!  See, that wasn't so hard!
+
+### Authentication Modes:
+The GoogleSheets plugin supports the `SHARED_USER` and `USER_TRANSLATION` authentication modes. `SHARED_USER` is as the name implies, one user for everyone. `USER_TRANSLATION` 

Review Comment:
   ```suggestion
   The Google Sheets plugin supports the `SHARED_USER` and `USER_TRANSLATION` authentication modes. `SHARED_USER` is as the name implies, one user for everyone. `USER_TRANSLATION` 
   ```



##########
contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.googlesheets;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.categories.RowSetTest;
+import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.exec.oauth.PersistentTokenTable;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder.QuerySummary;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category(RowSetTest.class)
+@Ignore("These tests require a live Google Sheets connection.  Please run manually.")
+public class TestGoogleSheetsWriter extends ClusterTest {
+  private static final String AUTH_URI = "https://accounts.google.com/o/oauth2/auth";
+  private static final String TOKEN_URI = "https://oauth2.googleapis.com/token";
+  private static final List<String> REDIRECT_URI = new ArrayList<>(Arrays.asList("urn:ietf:wg:oauth:2.0:oob", "http://localhost"));
+
+  private static StoragePluginRegistry pluginRegistry;
+  private static String accessToken;
+  private static String refreshToken;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+    dirTestWatcher.copyResourceToRoot(Paths.get(""));
+
+    String oauthJson = Files.asCharSource(DrillFileUtils.getResourceAsFile("/tokens/oauth_tokens.json"), Charsets.UTF_8).read();
+
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String,String> tokenMap = mapper.readValue(oauthJson, Map.class);
+
+    String clientID = tokenMap.get("client_id");
+    String clientSecret = tokenMap.get("client_secret");
+    accessToken = tokenMap.get("access_token");
+    refreshToken = tokenMap.get("refresh_token");
+
+    pluginRegistry = cluster.drillbit().getContext().getStorage();
+    GoogleSheetsStoragePluginConfig config = GoogleSheetsStoragePluginConfig.builder()
+      .clientID(clientID)
+      .clientSecret(clientSecret)
+      .redirectUris(REDIRECT_URI)
+      .authUri(AUTH_URI)
+      .tokenUri(TOKEN_URI)
+      .build();
+
+    config.setEnabled(true);
+    pluginRegistry.validatedPut("googlesheets", config);
+  }
+
+  @Test
+  public void testBasicCTAS() throws Exception {
+    try {
+      initializeTokens();
+    } catch (PluginException e) {
+      fail(e.getMessage());
+    }
+
+    String query = "CREATE TABLE googlesheets.`test_sheet`.`test_table` (ID, NAME) AS " +
+      "SELECT * FROM (VALUES(1,2), (3,4))";
+    // Create the table and insert the values
+    QuerySummary insertResults = queryBuilder().sql(query).run();
+    assertTrue(insertResults.succeeded());
+  }
+
+  @Test
+  public void testCTASFromFile() throws Exception {
+    try {
+      initializeTokens();
+    } catch (PluginException e) {
+      fail(e.getMessage());
+    }
+
+    /*String query = "CREATE TABLE googlesheets.`test_sheet`.`test_table` (ID, NAME) AS " +
+      "SELECT * FROM (VALUES(1,2), (3,4))";*/
+   String sql = "SELECT * FROM table(cp.`data/Drill_Test_Data.xlsx` (type => 'excel', sheetName => 'MixedSheet'))";

Review Comment:
   Is this actually testing a CTAS at the moment? It also looks like some commented temporary code can be cleaned up.



##########
contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsPushDownListener.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.googlesheets;
+
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Pair;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.base.filter.ExprNode;
+import org.apache.drill.exec.store.base.filter.ExprNode.AndNode;
+import org.apache.drill.exec.store.base.filter.ExprNode.ColRelOpConstNode;
+import org.apache.drill.exec.store.base.filter.ExprNode.OrNode;
+import org.apache.drill.exec.store.base.filter.FilterPushDownListener;
+import org.apache.drill.exec.store.base.filter.FilterPushDownStrategy;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * The GoogleSheets storage plugin accepts filters which are:
+ * <ul>
+ * <li>A single column = value expression </li>
+ * <li>An AND'ed set of such expressions,</li>
+ * <li>If the value is one with an unambiguous conversion to
+ * a string. (That is, not dates, binary, maps, etc.)</li>
+ * </ul>
+ */
+public class GoogleSheetsPushDownListener implements FilterPushDownListener {
+
+  public static Set<StoragePluginOptimizerRule> rulesFor(OptimizerRulesContext optimizerRulesContext) {
+    return FilterPushDownStrategy.rulesFor(new GoogleSheetsPushDownListener());
+  }
+
+  @Override
+  public String prefix() {
+    return "GoogleSheets";
+  }
+
+  @Override
+  public boolean isTargetScan(GroupScan groupScan) {
+    return groupScan instanceof GoogleSheetsGroupScan;
+  }
+
+  @Override
+  public ScanPushDownListener builderFor(GroupScan groupScan) {
+    GoogleSheetsGroupScan gsScan = (GoogleSheetsGroupScan) groupScan;
+    if (gsScan.hasFilters() || !gsScan.allowsFilters()) {
+      return null;
+    } else {
+      return new GoogleSheetsScanPushDownListener(gsScan);
+    }
+  }
+
+  private static class GoogleSheetsScanPushDownListener implements ScanPushDownListener {
+
+    private final GoogleSheetsGroupScan groupScan;
+    private final Map<String, String> filterParams = CaseInsensitiveMap.newHashMap();
+
+    GoogleSheetsScanPushDownListener(GoogleSheetsGroupScan groupScan) {
+      this.groupScan = groupScan;
+      for (SchemaPath field : groupScan.columns()) {
+        filterParams.put(field.getAsUnescapedPath(), field.getAsUnescapedPath());
+      }
+    }
+
+    @Override
+    public ExprNode accept(ExprNode node) {
+      if (node instanceof OrNode) {
+        return null;
+      } else if (node instanceof ColRelOpConstNode) {
+        return null;
+        //return acceptRelOp((ColRelOpConstNode) node);
+      } else {
+        return null;
+      }

Review Comment:
   This conditional reduces to `return null`, is that what is intended?



##########
contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/utils/GoogleSheetsTypifier.java:
##########
@@ -0,0 +1,346 @@
+/*
+ * 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.googlesheets.utils;
+
+import java.nio.CharBuffer;
+
+import java.time.LocalDate;
+import java.time.format.DateTimeParseException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Map.Entry;
+import java.util.AbstractMap.SimpleEntry;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+
+/**
+ * This class attempts to infer the data type of an unknown data type. It is somewhat
+ * configurable.  This was sourced from <a href="https://gist.github.com/awwsmm/56b8164410c89c719ebfca7b3d85870b">this code on github</a>.

Review Comment:
   I went to check license compatibility but I see no copyright notice or license at the source Gist. I'm not sure if we need to confirm this?



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 
+  `http://localhost:8047/credentials/<plugin name>/update_oauth2_authtoken`.
+
+1. To obtain the `clientID` and `client_secret` you will need to obtain the Google keys, open the [Google Sheets API](https://console.cloud.google.com/apis/library/sheets.googleapis.com) and click on the `Enable` button. 
+2. Once you've enabled teh API, you will be taken to the API Manager.  Either select a pre-existing project or create a new one.
+3. Next, navigate to the `Credentials` in the left panel.
+4. Click on `+Create Credentials` at the top of the page.  Select `OAuth client ID` and select `Web Application` or `Desktop` as the type.  Follow the instructions and download 
+   the JSON file that Google provides.
+
+Drill does not use the JSON file, but you will be cutting and pasting values from the JSON file into the Drill configuration.
+
+## Setup Step 2:  Configure Drill
+Create a storage plugin following the normal procedure for doing so.  You can use the example below as a template.  Cut and paste the `clientID` and `client_secret` from the 
+JSON file into your Drill configuration as shown below.  Once you've done that, save the configuration.
+
+```json
+{
+  "type": "googlesheets",
+  "allTextMode": true,
+  "extractHeaders": true,
+  "oAuthConfig": {
+    "callbackURL": "http://localhost:8047/credentials/googlesheets/update_oauth2_authtoken",
+    "authorizationURL": "https://accounts.google.com/o/oauth2/auth",
+    "authorizationParams": {
+      "response_type": "code",
+      "scope": "https://www.googleapis.com/auth/spreadsheets"
+    }
+  },
+  "credentialsProvider": {
+    "credentialsProviderType": "PlainCredentialsProvider",
+    "credentials": {
+      "clientID": "<YOUR CLIENT ID>",
+      "clientSecret": "<YOUR CLIENT SECRET>",
+      "tokenURI": "https://oauth2.googleapis.com/token"
+    },
+    "userCredentials": {}
+  },
+  "enabled": true,
+  "authMode": "SHARED_USER"
+}
+```
+
+With the exception of the clientID, client_secret and redirects, you should not have to modify any of the other parameters in the configuration. 

Review Comment:
   ```suggestion
   With the exception of the clientID, clientSecret and redirects, you should not have to modify any of the other parameters in the configuration. 
   ```



##########
.gitignore:
##########
@@ -28,3 +28,6 @@ exec/jdbc-all/dependency-reduced-pom.xml
 .*.html
 venv/
 tools/venv/
+
+# Directory to store oauth tokens for testing Googlesheets Storage plugin

Review Comment:
   Let's start keeping plugins' .gitignore rules in their own new .gitignore files. E.g. this rule would go into a new contrib/storage-googlesheets/.gitignore file and the ignored path would become src/test/resources/tokens/*.



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 

Review Comment:
   ```suggestion
   * Your redirect URL:  This is the URL to which Google will send the various access tokens and which you will need later.  For a local installation of Drill, it will be: 
   ```



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 
+  `http://localhost:8047/credentials/<plugin name>/update_oauth2_authtoken`.
+
+1. To obtain the `clientID` and `client_secret` you will need to obtain the Google keys, open the [Google Sheets API](https://console.cloud.google.com/apis/library/sheets.googleapis.com) and click on the `Enable` button. 
+2. Once you've enabled teh API, you will be taken to the API Manager.  Either select a pre-existing project or create a new one.
+3. Next, navigate to the `Credentials` in the left panel.
+4. Click on `+Create Credentials` at the top of the page.  Select `OAuth client ID` and select `Web Application` or `Desktop` as the type.  Follow the instructions and download 
+   the JSON file that Google provides.
+
+Drill does not use the JSON file, but you will be cutting and pasting values from the JSON file into the Drill configuration.
+
+## Setup Step 2:  Configure Drill
+Create a storage plugin following the normal procedure for doing so.  You can use the example below as a template.  Cut and paste the `clientID` and `client_secret` from the 
+JSON file into your Drill configuration as shown below.  Once you've done that, save the configuration.
+
+```json
+{
+  "type": "googlesheets",
+  "allTextMode": true,
+  "extractHeaders": true,
+  "oAuthConfig": {
+    "callbackURL": "http://localhost:8047/credentials/googlesheets/update_oauth2_authtoken",
+    "authorizationURL": "https://accounts.google.com/o/oauth2/auth",
+    "authorizationParams": {
+      "response_type": "code",
+      "scope": "https://www.googleapis.com/auth/spreadsheets"
+    }
+  },
+  "credentialsProvider": {
+    "credentialsProviderType": "PlainCredentialsProvider",
+    "credentials": {
+      "clientID": "<YOUR CLIENT ID>",
+      "clientSecret": "<YOUR CLIENT SECRET>",
+      "tokenURI": "https://oauth2.googleapis.com/token"
+    },
+    "userCredentials": {}
+  },
+  "enabled": true,
+  "authMode": "SHARED_USER"
+}
+```
+
+With the exception of the clientID, client_secret and redirects, you should not have to modify any of the other parameters in the configuration. 
+
+### Other Configuration Parameters
+
+There are two configuration parameters which you may want to adjust:
+* `allTextMode`:  This parameter when `true` disables Drill's data type inferencing for your files.  If your data has inconsistent data types, set this to `true`.  Default is 
+  `true`. 
+* `extractHeaders`:  When `true`, Drill will treat the first row of your data as headers.  When `false` Drill will assign column names like `field_n` for each column.
+
+### Authenticating with Google
+Once you have configured Drill to query GoogleSheets, there is one final step before you can access data.  You must authenticate the application (Drill) with GoogleSheets.  After you have saved your GoogleSheets configuration, navigate back to the configuration screen for your plugin and click on `Authorize`. A new window should appear which will prompt you to authenticate with Google services.  Once you have done that, you should be able to query GoogleSheets!  See, that wasn't so hard!
+
+### Authentication Modes:
+The GoogleSheets plugin supports the `SHARED_USER` and `USER_TRANSLATION` authentication modes. `SHARED_USER` is as the name implies, one user for everyone. `USER_TRANSLATION` 
+uses different credentials for each individual user.  In this case, the credentials are the OAuth2.0 access tokens.  
+
+At the time of writing, we have not yet documented `USER_TRANSLATION` fully, however we will update this readme once that is complete.
+
+## Querying Data
+Once you have configured Drill to query 
+
+### Obtaining the SpreadsheetID
+The URL below is a public spreadsheet hosted on GoogleSheets:
+[https://docs.google.com/spreadsheets/d/1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms/](https://docs.google.com/spreadsheets/d/1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms/)
+
+In this URL, the portion `1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms` is the spreadsheetID. Thus, 
+if you wanted to query this sheet in Drill, after configuring Drill, you could do so with the following
+query:
+
+```sql
+SELECT * 
+FROM googlesheets.`1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms`.`Class Data`
+```
+
+The format for the `FROM` clause for GoogleSheets is:
+```sql
+FROM <plugin name>.<sheet ID>.<tab name>
+```
+Note that you must specify the tab name to successfully query GoogleSheets.
+
+### Using Aliases
+Since the sheet IDs from Google are not human readable, one way to make your life easier is to use Drill's aliasing features to provide a better name for the actual sheet name. 
+
+### Data Types
+Drill's Google Sheets reader will attempt to infer the data types of the incoming data.  As with other connectors, this is an imperfect process since GoogleSheets does not 
+supply a schema or other information to allow Drill to identify the data types of a column.  At present, here is how Drill will map your data:
+* Numbers:  All numeric columns will be mapped to `DOUBLE` data types
+* Boolean:  Columns containing `true/false` will be mapped to the `BOOLEAN` type
+* Time, Date, Timestamp:  Temporal fields will be mapped to the correct type.  You can disable able temporal fields by setting the config option `XXX` to `false`.

Review Comment:
   What is the config option `XXX` here?



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 
+  `http://localhost:8047/credentials/<plugin name>/update_oauth2_authtoken`.
+
+1. To obtain the `clientID` and `client_secret` you will need to obtain the Google keys, open the [Google Sheets API](https://console.cloud.google.com/apis/library/sheets.googleapis.com) and click on the `Enable` button. 
+2. Once you've enabled teh API, you will be taken to the API Manager.  Either select a pre-existing project or create a new one.

Review Comment:
   ```suggestion
   2. Once you've enabled the API you will be taken to the API Manager.  Either select an existing project or create a new one.
   ```



##########
exec/java-exec/src/main/java/org/apache/drill/exec/store/security/OAuthTokenCredentials.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.security;
+
+import org.apache.drill.common.logical.security.CredentialsProvider;
+
+import java.util.Map;
+
+public class OAuthTokenCredentials {

Review Comment:
   We do already have org.apache.drill.exec.store.security.oauth.OAuthTokenCredentials, possible duplicate?



##########
contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsGroupScan.java:
##########
@@ -0,0 +1,369 @@
+/*
+ * 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.googlesheets;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.metastore.MetadataProviderManager;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.base.filter.ExprNode;
+import org.apache.drill.exec.util.Utilities;
+import org.apache.drill.metastore.metadata.TableMetadata;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+@JsonTypeName("googlesheets-group-scan")
+public class GoogleSheetsGroupScan extends AbstractGroupScan {
+
+  private final GoogleSheetsScanSpec scanSpec;
+  private final GoogleSheetsStoragePluginConfig config;
+  private final List<SchemaPath> columns;
+  private final String pluginName;
+  private final Map<String, ExprNode.ColRelOpConstNode> filters;
+  private final ScanStats scanStats;
+  private final double filterSelectivity;
+  private final int maxRecords;
+  private final GoogleSheetsStoragePlugin plugin;
+  private int hashCode;
+  private MetadataProviderManager metadataProviderManager;
+
+  // Initial Constructor
+  public GoogleSheetsGroupScan(String userName,
+                               GoogleSheetsScanSpec scanSpec,
+                               GoogleSheetsStoragePlugin plugin,
+                               MetadataProviderManager metadataProviderManager) {
+    super(userName);
+    this.scanSpec = scanSpec;
+    this.config = scanSpec.getConfig();
+    this.columns = ALL_COLUMNS;
+    this.pluginName = plugin.getName();
+    this.filters = null;
+    this.filterSelectivity = 0.0;
+    this.maxRecords = -1;
+    this.scanStats = computeScanStats();
+    this.plugin = plugin;
+    this.metadataProviderManager = metadataProviderManager;
+  }
+
+  // Copy Constructor
+  public GoogleSheetsGroupScan(GoogleSheetsGroupScan that) {
+    super(that);
+    this.scanSpec = that.scanSpec;
+    this.config = that.config;
+    this.columns = that.columns;
+    this.filters = that.filters;
+    this.pluginName = that.pluginName;
+    this.filterSelectivity = that.filterSelectivity;
+    this.scanStats = that.scanStats;
+    this.maxRecords = that.maxRecords;
+    this.plugin = that.plugin;
+    this.metadataProviderManager = that.metadataProviderManager;
+    this.hashCode = hashCode();
+  }
+
+  /**
+   * Constructor for applying a limit.
+   * @param that The previous group scan without the limit.
+   * @param maxRecords  The desired limit, pushed down from Calcite
+   */
+  public GoogleSheetsGroupScan(GoogleSheetsGroupScan that, int maxRecords) {
+    super(that);
+    this.scanSpec = that.scanSpec;
+    this.config = that.config;
+    this.columns = that.columns;
+    this.pluginName = that.pluginName;
+    this.filters = that.filters;
+    this.filterSelectivity = that.filterSelectivity;
+    this.maxRecords = maxRecords;
+    this.plugin = that.plugin;
+    this.metadataProviderManager = that.metadataProviderManager;
+    this.scanStats = computeScanStats();
+  }
+
+  /**
+   * Constructor for applying columns (Projection pushdown).
+   * @param that The previous GroupScan, without the columns
+   * @param columns The list of columns to push down
+   */
+  public GoogleSheetsGroupScan(GoogleSheetsGroupScan that, List<SchemaPath> columns) {
+    super(that);
+    this.scanSpec = that.scanSpec;
+    this.config = scanSpec.getConfig();
+    this.columns = columns;
+    this.filters = that.filters;
+    this.pluginName = that.pluginName;
+    this.filterSelectivity = that.filterSelectivity;
+    this.maxRecords = that.maxRecords;
+    this.plugin = that.plugin;
+    this.metadataProviderManager = that.metadataProviderManager;
+    this.scanStats = computeScanStats();
+  }
+
+  /**
+   * Constructor for applying a filter
+   * @param that Previous group scan w/o filters
+   * @param filters The list of filters
+   * @param filterSelectivity  The filter selectivity
+   */
+  public GoogleSheetsGroupScan(GoogleSheetsGroupScan that,
+                               Map<String, ExprNode.ColRelOpConstNode> filters,
+                               double filterSelectivity) {
+    super(that);
+    this.scanSpec = that.scanSpec;
+    this.config = that.config;
+    this.columns = that.columns;
+    this.filters = filters;
+    this.pluginName = that.pluginName;
+    this.filterSelectivity = filterSelectivity;
+    this.maxRecords = that.maxRecords;
+    this.plugin = that.plugin;
+    this.metadataProviderManager = that.metadataProviderManager;
+    this.scanStats = computeScanStats();
+  }
+
+  @JsonCreator
+  public GoogleSheetsGroupScan(
+    @JsonProperty("userName") String userName,
+    @JsonProperty("scanSpec") GoogleSheetsScanSpec scanSpec,
+    @JsonProperty("columns") List<SchemaPath> columns,
+    @JsonProperty("filters") Map<String, ExprNode.ColRelOpConstNode> filters,
+    @JsonProperty("filterSelectivity") double selectivity,
+    @JsonProperty("maxRecords") int maxRecords,
+    @JacksonInject StoragePluginRegistry plugins
+  ) {
+    super(userName);
+    this.scanSpec = scanSpec;
+    this.config = scanSpec.getConfig();
+    this.columns = columns;
+    this.filters = filters;
+    this.filterSelectivity = selectivity;
+    this.maxRecords = maxRecords;
+    this.scanStats = computeScanStats();
+    this.plugin = plugins.resolve(config, GoogleSheetsStoragePlugin.class);
+    this.pluginName = plugin.getName();
+  }
+
+  @JsonProperty("scanSpec")
+  public GoogleSheetsScanSpec scanSpec() {
+    return scanSpec;
+  }
+
+  @JsonProperty("config")
+  public GoogleSheetsStoragePluginConfig config() {
+    return config;
+  }
+
+  @JsonProperty("columns")
+  public List<SchemaPath> columns() {
+    return columns;
+  }
+
+  @JsonProperty("filters")
+  public Map<String, ExprNode.ColRelOpConstNode> filters() {
+    return filters;
+  }
+
+  @JsonProperty("maxRecords")
+  public int maxRecords() {
+    return maxRecords;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+
+  }
+
+  public TupleMetadata getSchema() {
+    if (metadataProviderManager == null) {
+      return null;
+    }
+    try {
+      return metadataProviderManager.getSchemaProvider().read().getSchema();
+    } catch (IOException | NullPointerException e) {
+      return null;
+    }
+  }
+
+  @Override
+  public TableMetadata getTableMetadata() {
+    if (getMetadataProvider() == null) {
+      return null;
+    }
+    return getMetadataProvider().getTableMetadata();
+  }
+
+  @Override
+  public TableMetadataProvider getMetadataProvider() {
+    if (metadataProviderManager == null) {
+      return null;
+    }
+    return metadataProviderManager.getTableMetadataProvider();
+  }
+
+  @Override
+  @JsonIgnore
+  public boolean canPushdownProjects(List<SchemaPath> columns) {
+    return true;
+  }
+
+  @JsonIgnore
+  public boolean allowsFilters() {
+    return true;
+  }
+
+  @Override
+  public SubScan getSpecificScan(int minorFragmentId) {
+    return new GoogleSheetsSubScan(userName, config, scanSpec, columns, filters, maxRecords, getSchema());
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    return 1;
+  }
+
+  @Override
+  public GroupScan clone(List<SchemaPath> columns) {
+    return new GoogleSheetsGroupScan(this, columns);
+  }
+
+  @Override
+  public boolean supportsLimitPushdown() {
+    return true;
+  }
+
+  @Override
+  public GroupScan applyLimit(int maxRecords) {
+    if (maxRecords == this.maxRecords) {
+      return null;
+    }
+    return new GoogleSheetsGroupScan(this, maxRecords);
+  }
+
+  @Override
+  public String getDigest() {
+    return toString();
+  }
+
+  @Override
+  public ScanStats getScanStats() {
+
+    // Since this class is immutable, compute stats once and cache
+    // them. If the scan changes (adding columns, adding filters), we
+    // get a new scan without cached stats.
+    return scanStats;
+  }
+
+  private ScanStats computeScanStats() {
+
+    // If this config allows filters, then make the default
+    // cost very high to force the planner to choose the version
+    // with filters.
+    if (!hasFilters()) {
+      return new ScanStats(ScanStats.GroupScanProperty.ESTIMATED_TOTAL_COST,
+        1E9, 1E112, 1E12);
+    }
+
+    // No good estimates at all, just make up something.
+    double estRowCount = 10_000;

Review Comment:
   It might be possible to get [this information from the Sheets API](https://developers.google.com/sheets/api/reference/rest/v4/spreadsheets/sheets#GridProperties) but how much benefit that would confer I do not know. Another option would be to leave a `// TODO: consider setting estRowCount using the sheet row count metadata from the Sheets API`.
   
   



##########
contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/columns/GoogleSheetsDateColumnWriter.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.googlesheets.columns;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+
+import java.time.LocalDate;
+
+public class GoogleSheetsDateColumnWriter extends GoogleSheetsColumnWriter {

Review Comment:
   Consider converting this (and siblings) to a static inner class GoogleSheetsColumnWriter.DateWriter to reduce the proliferation of small source files.



##########
contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsLimitPushdown.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.googlesheets;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.oauth.PersistentTokenTable;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.fail;
+
+@Ignore("This test requires a live connection to GoogleSheets.  Please run tests manually.")

Review Comment:
   ```suggestion
   @Ignore("This test requires a live connection to Google Sheets.  Please run tests manually.")
   ```



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 
+  `http://localhost:8047/credentials/<plugin name>/update_oauth2_authtoken`.
+
+1. To obtain the `clientID` and `client_secret` you will need to obtain the Google keys, open the [Google Sheets API](https://console.cloud.google.com/apis/library/sheets.googleapis.com) and click on the `Enable` button. 
+2. Once you've enabled teh API, you will be taken to the API Manager.  Either select a pre-existing project or create a new one.
+3. Next, navigate to the `Credentials` in the left panel.
+4. Click on `+Create Credentials` at the top of the page.  Select `OAuth client ID` and select `Web Application` or `Desktop` as the type.  Follow the instructions and download 
+   the JSON file that Google provides.
+
+Drill does not use the JSON file, but you will be cutting and pasting values from the JSON file into the Drill configuration.
+
+## Setup Step 2:  Configure Drill
+Create a storage plugin following the normal procedure for doing so.  You can use the example below as a template.  Cut and paste the `clientID` and `client_secret` from the 
+JSON file into your Drill configuration as shown below.  Once you've done that, save the configuration.
+
+```json
+{
+  "type": "googlesheets",
+  "allTextMode": true,
+  "extractHeaders": true,
+  "oAuthConfig": {
+    "callbackURL": "http://localhost:8047/credentials/googlesheets/update_oauth2_authtoken",
+    "authorizationURL": "https://accounts.google.com/o/oauth2/auth",
+    "authorizationParams": {
+      "response_type": "code",
+      "scope": "https://www.googleapis.com/auth/spreadsheets"
+    }
+  },
+  "credentialsProvider": {
+    "credentialsProviderType": "PlainCredentialsProvider",
+    "credentials": {
+      "clientID": "<YOUR CLIENT ID>",
+      "clientSecret": "<YOUR CLIENT SECRET>",
+      "tokenURI": "https://oauth2.googleapis.com/token"
+    },
+    "userCredentials": {}
+  },
+  "enabled": true,
+  "authMode": "SHARED_USER"
+}
+```
+
+With the exception of the clientID, client_secret and redirects, you should not have to modify any of the other parameters in the configuration. 
+
+### Other Configuration Parameters
+
+There are two configuration parameters which you may want to adjust:
+* `allTextMode`:  This parameter when `true` disables Drill's data type inferencing for your files.  If your data has inconsistent data types, set this to `true`.  Default is 
+  `true`. 
+* `extractHeaders`:  When `true`, Drill will treat the first row of your data as headers.  When `false` Drill will assign column names like `field_n` for each column.
+
+### Authenticating with Google
+Once you have configured Drill to query GoogleSheets, there is one final step before you can access data.  You must authenticate the application (Drill) with GoogleSheets.  After you have saved your GoogleSheets configuration, navigate back to the configuration screen for your plugin and click on `Authorize`. A new window should appear which will prompt you to authenticate with Google services.  Once you have done that, you should be able to query GoogleSheets!  See, that wasn't so hard!
+
+### Authentication Modes:
+The GoogleSheets plugin supports the `SHARED_USER` and `USER_TRANSLATION` authentication modes. `SHARED_USER` is as the name implies, one user for everyone. `USER_TRANSLATION` 
+uses different credentials for each individual user.  In this case, the credentials are the OAuth2.0 access tokens.  
+
+At the time of writing, we have not yet documented `USER_TRANSLATION` fully, however we will update this readme once that is complete.
+
+## Querying Data
+Once you have configured Drill to query 
+
+### Obtaining the SpreadsheetID
+The URL below is a public spreadsheet hosted on GoogleSheets:
+[https://docs.google.com/spreadsheets/d/1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms/](https://docs.google.com/spreadsheets/d/1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms/)
+
+In this URL, the portion `1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms` is the spreadsheetID. Thus, 
+if you wanted to query this sheet in Drill, after configuring Drill, you could do so with the following
+query:
+
+```sql
+SELECT * 
+FROM googlesheets.`1BxiMVs0XRA5nFMdKvBdBZjgmUUqptlbs74OgvE2upms`.`Class Data`
+```
+
+The format for the `FROM` clause for GoogleSheets is:
+```sql
+FROM <plugin name>.<sheet ID>.<tab name>
+```
+Note that you must specify the tab name to successfully query GoogleSheets.
+
+### Using Aliases

Review Comment:
   Nice application of @vvysotskyi's aliases :)



##########
contrib/storage-googlesheets/README.md:
##########
@@ -0,0 +1,156 @@
+# Google Sheets Connector for Apache Drill
+This connector enables you to query and write to Google Sheets.  
+
+### Usage Notes:
+This feature should be considered experimental as Google's API for Sheets is quite complex and amazingly 
+poorly documented.
+
+## Setup Step 1:  Obtain Credential Information from Google
+Ok... this is a pain.  GoogleSheets uses OAuth2.0 (may it be quickly deprecated) for authorization. In order to query GoogleSheets, you will first need to obtain three artifacts:
+
+* Your `clientID`:  This is an identifier which uniquely identifies your application to Google
+* Your `client_secret`: You can think of this as your password for your application to access GoogleSheets
+* Your redirect URL:  This is the URL which Google will send the various access tokens which you will need later.  For a local installation of Drill, it will be: 
+  `http://localhost:8047/credentials/<plugin name>/update_oauth2_authtoken`.
+
+1. To obtain the `clientID` and `client_secret` you will need to obtain the Google keys, open the [Google Sheets API](https://console.cloud.google.com/apis/library/sheets.googleapis.com) and click on the `Enable` button. 
+2. Once you've enabled teh API, you will be taken to the API Manager.  Either select a pre-existing project or create a new one.
+3. Next, navigate to the `Credentials` in the left panel.
+4. Click on `+Create Credentials` at the top of the page.  Select `OAuth client ID` and select `Web Application` or `Desktop` as the type.  Follow the instructions and download 
+   the JSON file that Google provides.
+
+Drill does not use the JSON file, but you will be cutting and pasting values from the JSON file into the Drill configuration.
+
+## Setup Step 2:  Configure Drill
+Create a storage plugin following the normal procedure for doing so.  You can use the example below as a template.  Cut and paste the `clientID` and `client_secret` from the 
+JSON file into your Drill configuration as shown below.  Once you've done that, save the configuration.
+
+```json
+{
+  "type": "googlesheets",
+  "allTextMode": true,
+  "extractHeaders": true,
+  "oAuthConfig": {
+    "callbackURL": "http://localhost:8047/credentials/googlesheets/update_oauth2_authtoken",
+    "authorizationURL": "https://accounts.google.com/o/oauth2/auth",
+    "authorizationParams": {
+      "response_type": "code",
+      "scope": "https://www.googleapis.com/auth/spreadsheets"
+    }
+  },
+  "credentialsProvider": {
+    "credentialsProviderType": "PlainCredentialsProvider",
+    "credentials": {
+      "clientID": "<YOUR CLIENT ID>",
+      "clientSecret": "<YOUR CLIENT SECRET>",
+      "tokenURI": "https://oauth2.googleapis.com/token"
+    },
+    "userCredentials": {}
+  },
+  "enabled": true,
+  "authMode": "SHARED_USER"
+}
+```
+
+With the exception of the clientID, client_secret and redirects, you should not have to modify any of the other parameters in the configuration. 
+
+### Other Configuration Parameters
+
+There are two configuration parameters which you may want to adjust:
+* `allTextMode`:  This parameter when `true` disables Drill's data type inferencing for your files.  If your data has inconsistent data types, set this to `true`.  Default is 
+  `true`. 
+* `extractHeaders`:  When `true`, Drill will treat the first row of your data as headers.  When `false` Drill will assign column names like `field_n` for each column.
+
+### Authenticating with Google
+Once you have configured Drill to query GoogleSheets, there is one final step before you can access data.  You must authenticate the application (Drill) with GoogleSheets.  After you have saved your GoogleSheets configuration, navigate back to the configuration screen for your plugin and click on `Authorize`. A new window should appear which will prompt you to authenticate with Google services.  Once you have done that, you should be able to query GoogleSheets!  See, that wasn't so hard!
+
+### Authentication Modes:
+The GoogleSheets plugin supports the `SHARED_USER` and `USER_TRANSLATION` authentication modes. `SHARED_USER` is as the name implies, one user for everyone. `USER_TRANSLATION` 
+uses different credentials for each individual user.  In this case, the credentials are the OAuth2.0 access tokens.  
+
+At the time of writing, we have not yet documented `USER_TRANSLATION` fully, however we will update this readme once that is complete.
+
+## Querying Data
+Once you have configured Drill to query 

Review Comment:
   Truncated sentence?



##########
contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/utils/GoogleSheetsTypifier.java:
##########
@@ -0,0 +1,346 @@
+/*
+ * 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.googlesheets.utils;
+
+import java.nio.CharBuffer;
+
+import java.time.LocalDate;
+import java.time.format.DateTimeParseException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Map.Entry;
+import java.util.AbstractMap.SimpleEntry;
+
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+
+/**
+ * This class attempts to infer the data type of an unknown data type. It is somewhat
+ * configurable.  This was sourced from <a href="https://gist.github.com/awwsmm/56b8164410c89c719ebfca7b3d85870b">this code on github</a>.
+ */
+public class GoogleSheetsTypifier {

Review Comment:
   Should we consider a system wide type guessing util class? I.e. move this to somewhere like common/ and make all our character-based data formats decide a Drill type using the same logic?



##########
contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsStoragePlugin.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.googlesheets;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.api.client.auth.oauth2.StoredCredential;
+import com.google.api.client.util.store.DataStore;
+import com.google.api.services.sheets.v4.Sheets;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.logical.StoragePluginConfig.AuthMode;
+import org.apache.drill.exec.metastore.MetadataProviderManager;
+import org.apache.drill.exec.oauth.OAuthTokenProvider;
+import org.apache.drill.exec.oauth.PersistentTokenTable;
+import org.apache.drill.exec.oauth.TokenRegistry;
+import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.planner.PlannerPhase;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.SessionOptionManager;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
+import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.base.filter.FilterPushDownUtils;
+import org.apache.drill.exec.store.googlesheets.schema.GoogleSheetsSchemaFactory;
+import org.apache.drill.exec.store.googlesheets.utils.GoogleSheetsUtils;
+import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.util.List;
+import java.util.Set;
+
+public class GoogleSheetsStoragePlugin extends AbstractStoragePlugin {
+  private final static Logger logger = LoggerFactory.getLogger(GoogleSheetsStoragePlugin.class);
+  private final static String SHARED_USERNAME = "anonymous";
+  private final GoogleSheetsStoragePluginConfig config;
+  private final GoogleSheetsSchemaFactory schemaFactory;
+  private final OAuthTokenProvider tokenProvider;
+  private DataStore<StoredCredential> dataStore;
+  private Sheets service;
+  private TokenRegistry tokenRegistry;
+  private String username;
+
+
+  public GoogleSheetsStoragePlugin(GoogleSheetsStoragePluginConfig configuration, DrillbitContext context, String name) {
+    super(context, name);
+    this.config = configuration;
+    this.tokenProvider = context.getoAuthTokenProvider();
+    this.schemaFactory = new GoogleSheetsSchemaFactory(this);
+  }
+
+  public void initializeOauthTokenTable(SchemaConfig schemaConfig) {
+    // A word about how GoogleSheets (GS) handles authorization and authentication.
+    // GS uses OAuth 2.0 for authorization.
+    // The GS Sheets object is the client which interacts with the actual data, however
+    // it does not provide a straightforward way of passing credentials into this object.
+    // GS has three objects:  the credential, storedCredential, and the credential dataStore.
+    //
+    // The Credential Object
+    // The credential really should be called the applicationCredential or something like that, as
+    // it stores the OAuth credentials for the application such as the clientID, clientSecret
+    //
+    // The Stored Credential Object
+    // This object has no relation to the Credential object, and it stores the user's credentials,
+    // specifically the access and refresh tokens.
+    //
+    // The DataStore Object is a synchronized store of storedCredential objects.
+    // The approach we take here is to use Drill's existing OAuth infrastructure
+    // to store the tokens in PersistentTokenStores, just like the HTTP plugin. When
+    // the plugin is loaded, we read the tokens from the persistent store into a GS dataStore.
+    // This happens when the plugin is registered.
+
+    if (config.getAuthMode() == AuthMode.USER_TRANSLATION) {
+      this.username = schemaConfig.getUserName();
+      tokenRegistry = tokenProvider.getOauthTokenRegistry(this.username);
+    } else {
+      this.username = SHARED_USERNAME;
+      tokenRegistry = tokenProvider.getOauthTokenRegistry(null);
+    }
+    tokenRegistry.createTokenTable(getName());
+    this.dataStore = new DrillDataStoreFactory(tokenProvider, getName()).createDataStore(this.username);
+  }
+
+  public DataStore<StoredCredential> getDataStore(String username) {
+    if (this.dataStore == null) {
+      this.dataStore = new DrillDataStoreFactory(tokenProvider, getName()).createDataStore(username);
+    }
+    return dataStore;
+  }
+
+
+  @Override
+  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) {
+    initializeOauthTokenTable(schemaConfig);
+    schemaFactory.registerSchemas(schemaConfig, parent);
+  }
+
+  public PersistentTokenTable getTokenTable() {
+    return tokenRegistry.getTokenTable(getName());
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection,
+                                           SessionOptionManager options) throws IOException {
+    return getPhysicalScan(userName, selection, AbstractGroupScan.ALL_COLUMNS,
+      options, null);
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection,
+                                           SessionOptionManager options, MetadataProviderManager metadataProviderManager) throws IOException {
+    return getPhysicalScan(userName, selection, AbstractGroupScan.ALL_COLUMNS,
+      options, metadataProviderManager);
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection,
+                                           List<SchemaPath> columns) throws IOException {
+    return getPhysicalScan(userName, selection, columns, null, null);
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
+    return getPhysicalScan(userName, selection, AbstractGroupScan.ALL_COLUMNS, null);
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns, SessionOptionManager options,
+                                           MetadataProviderManager metadataProviderManager) throws IOException {
+    GoogleSheetsScanSpec scanSpec = selection.getListWith(context.getLpPersistence().getMapper(), new TypeReference<GoogleSheetsScanSpec>() {});
+    return new GoogleSheetsGroupScan(this.username, scanSpec, this, metadataProviderManager);
+  }
+
+  @Override
+  public Set<? extends RelOptRule> getOptimizerRules(OptimizerRulesContext optimizerContext, PlannerPhase phase) {
+
+    // Push-down planning is done at the logical phase so it can
+    // influence parallelization in the physical phase. Note that many
+    // existing plugins perform filter push-down at the physical
+    // phase, which also works fine if push-down is independent of
+    // parallelization.
+    if (FilterPushDownUtils.isFilterPushDownPhase(phase) || phase == PlannerPhase.LOGICAL) {

Review Comment:
   ```suggestion
       if (FilterPushDownUtils.isFilterPushDownPhase(phase) {
   ```
   Does this method not already match the wanted logical planning phases?



##########
contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsQueries.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.googlesheets;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.oauth.PersistentTokenTable;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistry.PluginException;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.time.LocalDate;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * This class tests the Google Sheets plugin. Since GoogleSheets is essentially an API, these tests
+ * must be run with a live internet connection.  These tests use test data which can be found in the
+ * resources directory.
+ */
+@Ignore("Requires live connection to GoogleSheets.  Please run tests manually.")
+public class TestGoogleSheetsQueries extends ClusterTest {
+
+  private static final String AUTH_URI = "https://accounts.google.com/o/oauth2/auth";
+  private static final String TOKEN_URI = "https://oauth2.googleapis.com/token";
+  private static final List<String> REDIRECT_URI = new ArrayList<>(Arrays.asList("urn:ietf:wg:oauth:2.0:oob", "http://localhost"));
+
+  private static StoragePluginRegistry pluginRegistry;
+  private static String accessToken;
+  private static String refreshToken;
+  private static String sheetID;
+  private static String clientID;
+  private static String clientSecret;
+
+  @BeforeClass
+  public static void init() throws Exception {
+
+    String oauthJson = Files.asCharSource(DrillFileUtils.getResourceAsFile("/tokens/oauth_tokens.json"), Charsets.UTF_8).read();
+
+    ObjectMapper mapper = new ObjectMapper();
+    Map<String,String> tokenMap = mapper.readValue(oauthJson, Map.class);
+
+    clientID = tokenMap.get("client_id");
+    clientSecret = tokenMap.get("client_secret");
+    accessToken = tokenMap.get("access_token");
+    refreshToken = tokenMap.get("refresh_token");
+    sheetID = tokenMap.get("sheet_id");
+
+    ClusterFixtureBuilder builder = new ClusterFixtureBuilder(dirTestWatcher)
+      .configProperty(ExecConstants.HTTP_ENABLE, true)
+      .configProperty(ExecConstants.HTTP_PORT_HUNT, true)
+      .configProperty(ExecConstants.IMPERSONATION_ENABLED, true);
+
+    startCluster(builder);
+
+    pluginRegistry = cluster.drillbit().getContext().getStorage();
+    GoogleSheetsStoragePluginConfig config = GoogleSheetsStoragePluginConfig.builder()
+      .clientID(clientID)
+      .clientSecret(clientSecret)
+      .redirectUris(REDIRECT_URI)
+      .authUri(AUTH_URI)
+      .tokenUri(TOKEN_URI)
+      .allTextMode(false)
+      .extractHeaders(true)
+      .build();
+    config.setEnabled(true);
+    pluginRegistry.validatedPut("googlesheets", config);
+  }
+
+  @Test
+  public void testStarQuery() throws Exception {
+    try {
+      initializeTokens("googlesheets");
+    } catch (PluginException e) {
+      fail(e.getMessage());
+    }
+
+    String sql = String.format("SELECT * FROM googlesheets.`%s`.`MixedSheet` WHERE `Col2` < 6.0", sheetID);
+    RowSet results = queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("Col1", MinorType.VARCHAR)
+      .addNullable("Col2", MinorType.FLOAT8)
+      .addNullable("Col3", MinorType.DATE)
+      .buildSchema();
+
+   RowSet expected = client.rowSetBuilder(expectedSchema)
+      .addRow("Rosaline  Thales", 1.0, null)
+      .addRow("Abdolhossein  Detlev", 2.0001, LocalDate.parse("2020-04-30"))
+      .addRow(null, 4.0, LocalDate.parse("2020-06-30"))
+      .addRow("Yunus  Elena", 3.5, LocalDate.parse("2021-01-15"))
+      .addRow("Swaran  Ohiyesa", -63.8, LocalDate.parse("2021-04-08"))
+      .addRow("Kalani  Godabert", 0.0, LocalDate.parse("2021-06-28"))
+      .addRow("Caishen  Origenes", 5.0E-7, LocalDate.parse("2021-07-09"))
+      .addRow("Toufik  Gurgen", 2.0, LocalDate.parse("2021-11-05"))
+      .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testProjectPushdown() throws Exception {

Review Comment:
   Isn't it necessary to look in the plan string for the projected columns to test the pushdown part? I would expect this test to pass even if there was no projection pushdown...



##########
contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchReader.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.googlesheets;
+
+import com.google.api.services.sheets.v4.Sheets;
+import com.google.api.services.sheets.v4.model.Sheet;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;

Review Comment:
   ```suggestion
   import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
   ```
   Can we use the EVF2 API here?



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