You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by cg...@apache.org on 2023/02/08 20:29:58 UTC

[drill] branch master updated: DRILL-8395: Add Support for Insert and Drop Table to GoogleSheets Plugin (#2748)

This is an automated email from the ASF dual-hosted git repository.

cgivre pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new 2240ed8033 DRILL-8395: Add Support for Insert and Drop Table to GoogleSheets Plugin (#2748)
2240ed8033 is described below

commit 2240ed8033315d2737dfebf16daa494d6d134564
Author: Charles S. Givre <cg...@apache.org>
AuthorDate: Wed Feb 8 15:29:50 2023 -0500

    DRILL-8395: Add Support for Insert and Drop Table to GoogleSheets Plugin (#2748)
---
 contrib/storage-googlesheets/README.md             | 43 +++++++++----
 .../GoogleSheetsBatchInsertWriter.java             | 52 ++++++++++++++++
 .../googlesheets/GoogleSheetsBatchWriter.java      | 35 ++++++-----
 .../googlesheets/GoogleSheetsInsertWriter.java     | 57 +++++++++++++++++
 .../GoogleSheetsInsertWriterBatchCreator.java      | 42 +++++++++++++
 .../googlesheets/GoogleSheetsStoragePlugin.java    |  5 ++
 .../schema/GoogleSheetsDrillSchema.java            | 47 ++++++++++++--
 .../schema/GoogleSheetsRootSchema.java             | 32 ++++++++++
 .../googlesheets/utils/GoogleSheetsUtils.java      | 61 +++++++++++++++++-
 .../store/googlesheets/TestGoogleSheetsWriter.java | 72 +++++++++++++++++++++-
 .../src/test/resources/data/insert_data.csvh       |  3 +
 .../src/test/resources/data/insert_data2.csvh      |  3 +
 .../drill/exec/planner/sql/SchemaUtilities.java    |  1 +
 13 files changed, 419 insertions(+), 34 deletions(-)

diff --git a/contrib/storage-googlesheets/README.md b/contrib/storage-googlesheets/README.md
index 5f26582c99..38aaae6ad4 100644
--- a/contrib/storage-googlesheets/README.md
+++ b/contrib/storage-googlesheets/README.md
@@ -110,6 +110,12 @@ FROM `INFORMATION_SCHEMA`.`SCHEMATA`
 WHERE SCHEMA_NAME LIKE 'googlesheets%'
 ```
 
+### Implicit Metadata Fields
+GoogleSheets has two implicit metadata fields which are:
+
+* `_sheets`: This will return a list of sheet (tab) names in a given GS document
+* `_title`: You can also access the file name with the `_title` field.  Note that the file name is NOT unique and should not be used for querying data.
+
 Due to rate limits from Google, the tabs are not reported to the `INFORMATION_SCHEMA`.  However, it is possible to obtain a list of all available tabs with the following query:
 
 ```sql
@@ -118,8 +124,6 @@ FROM googlesheets.`<token>`.`<sheet>`
 LIMIT 1
 ```
 
-You can also access the file name with the `_title` field.  Note that the file name is NOT
-unique and should not be used for querying data.
 
 ### 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.
@@ -150,19 +154,36 @@ When Drill reads Google Sheets, it is assumed that the first row contains column
 If this is incorrect you can set the `extractHeaders` parameter to `false`and Drill will name each field `field_n` where `n` is the column index.
 
 # Writing Data To Google Sheets
-When Drill is connected to Google Sheets, you can also write data to Google Sheets. The basic procedure is
+When Drill is connected to GoogleSheets, you can also write data to Google Sheets. The basic procedure is
 the same as with any other data source.  Simply write a `CREATE TABLE AS` (CTAS) query and your data will be
-written to Google Sheets.
+written to GoogleSheets.
+
+If you use a GoogleSheets filetoken in your CTAS query, Drill will create a new tab in that GoogleSheets document. However, if you use a file name, Drill will create a new GoogleSheets document and then create a new tab within that document.  
+
+```sql
+-- This will add a tab to an existing GoogleSheets Document
+CREATE TABLE googlesheets.`2384r7wuf2934iroeci2390ue2ur3r23948230948`.`tab_name` AS SELECT * FROM data
+
+-- This will create a new GoogleSheets Document with a single tab
+CREATE TABLE googlesheets.`new_doc`.`tab1` AS SELECT * FROM data
+
+```
+
+### Inserting (Appending) to Existing GoogleSheets Tabs
+GoogleSheets also supports inserting (appending) data to existing GoogleSheets tabs.  Syntax is:
+
+```sql
+INSERT INTO googlesheets.`<file_token>`.`<tab name>` SELECT * FROM data
+```
 
-One challenge is that once you have created the new sheet, you will have to manually retrieve the spreadsheet ID
-from Google Sheets in order to query your new data.
 
 ### Dropping Tables
-At the time of implementation, it is only possible to delete tables from within a Google Sheets document. You may encounter errors if you try to delete tables from documents
-that only have one table in them.  The format for deleting a table is:
+The `DROP TABLE` command will drop a tab from a GoogleSheet document.  If the document only has one tab, the entire document will be deleted.
+
+The format for deleting a table is:
 
 ```sql
-DROP TABLE googlesheets.<sheet id>.<tab name>
+DROP TABLE googlesheets.<file_token>.<tab name>
 ```
 
 # Possible Future Work
@@ -173,10 +194,6 @@ requires you to use a non-human readable ID to identify the Sheet.  One possible
 much more usable would be to automatically create an alias (either public) automatically mapping the unreadable sheetID to the document title.
 This could be accomplished after the first query or after a CTAS query.
 
-### Google Drive Integration
-Integrating with Google Drive may allow additional functionality such as getting the actual document name, deleting documents and a few other basic functions. However, the
-Google Drive permissions require additional validation from Google.
-
 ### Additional Pushdowns
 The current implementation supports pushdowns for projection and limit.
 The Google Sheets API is quite complex and incredibly poorly documented. In this author's opinion, it is quite possibly one of the worst APIs he has ever seen.
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchInsertWriter.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchInsertWriter.java
new file mode 100644
index 0000000000..952557ad29
--- /dev/null
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchInsertWriter.java
@@ -0,0 +1,52 @@
+/*
+ * 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.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.store.googlesheets.utils.GoogleSheetsUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class GoogleSheetsBatchInsertWriter extends GoogleSheetsBatchWriter {
+  private static final Logger logger = LoggerFactory.getLogger(GoogleSheetsBatchInsertWriter.class);
+
+  public GoogleSheetsBatchInsertWriter(OperatorContext context, String name, GoogleSheetsWriter config) {
+    super(context, name, config);
+  }
+
+  @Override
+  public void updateSchema(VectorAccessible batch) {
+    // no-op
+  }
+
+  @Override
+  public void cleanup() {
+    try {
+      GoogleSheetsUtils.appendDataToGoogleSheet(service, sheetName, tabName, values);
+    } catch (IOException e) {
+      throw UserException.dataWriteError(e)
+          .message("Error writing to GoogleSheets " + e.getMessage())
+          .build(logger);
+    }
+  }
+}
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchWriter.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchWriter.java
index 6741a31c7d..78e737525e 100644
--- a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchWriter.java
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsBatchWriter.java
@@ -69,12 +69,12 @@ import java.util.Map;
 public class GoogleSheetsBatchWriter extends AbstractRecordWriter {
   private static final Logger logger = LoggerFactory.getLogger(GoogleSheetsBatchWriter.class);
 
-  private final Sheets service;
-  private final String tabName;
-  private final String sheetName;
-  private final List<List<Object>> values;
+  protected final Sheets service;
+  protected final String tabName;
+  protected final String sheetName;
+  protected final List<List<Object>> values;
   private List<Object> rowList;
-  private String spreadsheetID;
+  protected String spreadsheetID;
 
   public GoogleSheetsBatchWriter(OperatorContext context, String name, GoogleSheetsWriter config) {
     GoogleSheetsStoragePlugin plugin = config.getPlugin();
@@ -100,17 +100,24 @@ public class GoogleSheetsBatchWriter extends AbstractRecordWriter {
 
   @Override
   public void updateSchema(VectorAccessible batch) throws IOException {
-    // Create the new GoogleSheet doc
-    Spreadsheet spreadsheet = new Spreadsheet()
-      .setProperties(new SpreadsheetProperties().setTitle(sheetName));
 
-    spreadsheet = service.spreadsheets().create(spreadsheet)
-      .setFields("spreadsheetId")
-      .execute();
+    // If the incoming sheetName is actually a file token then simply add a new tab to the existing document.
+    if (GoogleSheetsUtils.isProbableFileToken(sheetName)) {
+      GoogleSheetsUtils.addTabToGoogleSheet(service, sheetName, tabName);
+      spreadsheetID = sheetName;
+    } else {
+      // Otherwise, create the new GoogleSheet document and add a tab.
+      Spreadsheet spreadsheet = new Spreadsheet()
+          .setProperties(new SpreadsheetProperties().setTitle(sheetName));
 
-    this.spreadsheetID = spreadsheet.getSpreadsheetId();
-    // Now add the tab
-    GoogleSheetsUtils.addTabToGoogleSheet(service, spreadsheetID, tabName);
+      spreadsheet = service.spreadsheets().create(spreadsheet)
+          .setFields("spreadsheetId")
+          .execute();
+
+      this.spreadsheetID = spreadsheet.getSpreadsheetId();
+      // Now add the tab
+      GoogleSheetsUtils.addTabToGoogleSheet(service, spreadsheetID, tabName);
+    }
 
     // Add the column names to the values list.  GoogleSheets does not have any concept
     // of column names, so we just insert the column names as the first row of data.
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsInsertWriter.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsInsertWriter.java
new file mode 100644
index 0000000000..6c5eedfb9b
--- /dev/null
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsInsertWriter.java
@@ -0,0 +1,57 @@
+/*
+ * 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.JsonProperty;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+
+public class GoogleSheetsInsertWriter extends GoogleSheetsWriter {
+
+  public static final String OPERATOR_TYPE = "GOOGLESHEETS_INSERT_WRITER";
+
+  @JsonCreator
+  public GoogleSheetsInsertWriter(
+      @JsonProperty("child") PhysicalOperator child,
+      @JsonProperty("sheetName") String sheetName,
+      @JsonProperty("name") String name,
+      @JsonProperty("storage") StoragePluginConfig storageConfig,
+      @JsonProperty("queryUser") String queryUser,
+      @JacksonInject StoragePluginRegistry engineRegistry) {
+    super(child, sheetName, name, storageConfig, queryUser, engineRegistry);
+  }
+
+  public GoogleSheetsInsertWriter(PhysicalOperator child, String sheetName, String name, String queryUser, GoogleSheetsStoragePlugin plugin) {
+    super(child, sheetName, name, queryUser, plugin);
+  }
+
+  @Override
+  public String getOperatorType() {
+    return OPERATOR_TYPE;
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new GoogleSheetsInsertWriter(child, getSheetName(), getTableName(), getQueryUser(), getPlugin());
+  }
+}
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsInsertWriterBatchCreator.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsInsertWriterBatchCreator.java
new file mode 100644
index 0000000000..e2314c81c9
--- /dev/null
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsInsertWriterBatchCreator.java
@@ -0,0 +1,42 @@
+/*
+ * 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.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.InsertWriterRecordBatch;
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+
+import java.util.List;
+
+@SuppressWarnings("unused")
+public class GoogleSheetsInsertWriterBatchCreator implements BatchCreator<GoogleSheetsInsertWriter> {
+  @Override
+  public CloseableRecordBatch getBatch(ExecutorFragmentContext context, GoogleSheetsInsertWriter config, List<RecordBatch> children) {
+    assert children != null && children.size() == 1;
+
+    UserCredentials userCreds = context.getContextInformation().getQueryUserCredentials();
+
+    return new InsertWriterRecordBatch(config, children.iterator().next(), context,
+        new GoogleSheetsBatchInsertWriter(null, config.getTableName(), config)
+    );
+  }
+}
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsStoragePlugin.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsStoragePlugin.java
index f7e7cc54aa..eba000374b 100644
--- a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsStoragePlugin.java
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/GoogleSheetsStoragePlugin.java
@@ -191,6 +191,11 @@ public class GoogleSheetsStoragePlugin extends AbstractStoragePlugin {
     return true;
   }
 
+  @Override
+  public boolean supportsInsert() {
+    return true;
+  }
+
   /**
    * This method gets (and caches) the Google Service needed for API calls.
    * @return An authenticated {@link Sheets} Google Sheets service.
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsDrillSchema.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsDrillSchema.java
index 911f74bbf8..462f85b206 100644
--- a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsDrillSchema.java
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsDrillSchema.java
@@ -18,6 +18,7 @@
 
 package org.apache.drill.exec.store.googlesheets.schema;
 
+import com.google.api.services.drive.Drive;
 import com.google.api.services.sheets.v4.Sheets;
 import com.google.api.services.sheets.v4.model.Sheet;
 import org.apache.calcite.schema.Table;
@@ -27,9 +28,11 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Writer;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
+import org.apache.drill.exec.planner.logical.ModifyTableEntry;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StorageStrategy;
+import org.apache.drill.exec.store.googlesheets.GoogleSheetsInsertWriter;
 import org.apache.drill.exec.store.googlesheets.GoogleSheetsScanSpec;
 import org.apache.drill.exec.store.googlesheets.GoogleSheetsStoragePlugin;
 import org.apache.drill.exec.store.googlesheets.GoogleSheetsStoragePluginConfig;
@@ -64,15 +67,17 @@ public class GoogleSheetsDrillSchema extends AbstractSchema {
   private final String fileToken;
   private final String fileName;
 
+  private List<Sheet> tabList;
+
   public GoogleSheetsDrillSchema(AbstractSchema parent, String fileToken,
                                  GoogleSheetsStoragePlugin plugin,
                                  SchemaConfig schemaConfig,
                                  Sheets sheetsService, String fileName) {
-    super(parent.getSchemaPath(), fileToken);
+    super(parent.getSchemaPath(), GoogleSheetsRootSchema.getFileTokenWithCorrectCase(((GoogleSheetsRootSchema) parent).getTokenMap(), fileToken));
     this.plugin = plugin;
     this.schemaConfig = schemaConfig;
-    this.fileToken = fileToken;
     this.parent = (GoogleSheetsRootSchema) parent;
+    this.fileToken = GoogleSheetsRootSchema.getFileTokenWithCorrectCase(((GoogleSheetsRootSchema) parent).getTokenMap(), fileToken);
     this.sheetsService = sheetsService;
     this.tableList = new ArrayList<>();
     this.fileName = fileName;
@@ -145,12 +150,11 @@ public class GoogleSheetsDrillSchema extends AbstractSchema {
   }
 
   private void populateActiveTables() {
-    List<Sheet> tabList;
     try {
       tabList = GoogleSheetsUtils.getTabList(sheetsService, fileToken);
     } catch (IOException e) {
       throw UserException.connectionError(e)
-        .message("Unable to obtain tab list for Google Sheet document " + fileToken)
+        .message("Unable to obtain tab list for Google Sheet document " + fileToken + ". " + e.getMessage())
         .build(logger);
     }
     // Add sub schemas to list, then create tables
@@ -178,7 +182,7 @@ public class GoogleSheetsDrillSchema extends AbstractSchema {
         .message(plugin.getName() + " is not writable.")
         .build(logger);
     }
-    String documentName = this.name;
+    String documentName = this.fileToken;
     return new CreateTableEntry() {
       @Override
       public Writer getWriter(PhysicalOperator child) {
@@ -192,6 +196,39 @@ public class GoogleSheetsDrillSchema extends AbstractSchema {
     };
   }
 
+  @Override
+  public ModifyTableEntry modifyTable(String tableName) {
+    return child -> new GoogleSheetsInsertWriter(child, this.fileToken, tableName, schemaConfig.getUserName(), plugin);
+  }
+
+  @Override
+  public void dropTable(String indexName) {
+    logger.debug("Index name: {}", indexName);
+
+    // The GoogleSheets API will not allow you to delete a tab if the file only has one tab.  In that case,
+    // we delete the entire file.
+    if (tabList.size() == 1) {
+      Drive driveService = plugin.getDriveService(schemaConfig.getUserName());
+      try {
+        driveService.files().delete(fileToken);
+      } catch (IOException e) {
+        throw UserException.internalError(e)
+            .message("Error deleting GoogleSheets file. " + e.getMessage())
+            .build(logger);
+      }
+    }
+
+    Sheet sheetToDrop = GoogleSheetsUtils.getSheetFromTabList(indexName, tabList);
+    try {
+      GoogleSheetsUtils.removeTabFromGoogleSheet(sheetsService, fileToken, sheetToDrop);
+    } catch (IOException e) {
+      throw UserException.internalError(e)
+          .message(e.getMessage())
+          .build(logger);
+    }
+  }
+
+
   private void registerTable(String name, DynamicDrillTable table) {
     activeTables.put(name, table);
     tableList.add(table);
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsRootSchema.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsRootSchema.java
index 0c00a7ed5e..9ceeb2f648 100644
--- a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsRootSchema.java
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/schema/GoogleSheetsRootSchema.java
@@ -21,6 +21,7 @@ package org.apache.drill.exec.store.googlesheets.schema;
 import com.google.api.services.drive.Drive;
 import com.google.api.services.sheets.v4.Sheets;
 import org.apache.calcite.schema.Table;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -67,6 +68,11 @@ public class GoogleSheetsRootSchema extends AbstractSchema {
 
   @Override
   public AbstractSchema getSubSchema(String name) {
+    // If the name is a file token but not mixed case, get the correct file token
+    if (GoogleSheetsUtils.isProbableFileToken(name) && !StringUtils.isMixedCase(name)) {
+      name = getFileTokenWithCorrectCase(tokenMap, name);
+    }
+
     GoogleSheetsDrillSchema schema = schemas.get(name);
     // This level here represents the actual Google document. Attempt to validate that it exists, and
     // if so, add it to the schema list.  If not, throw an exception.
@@ -102,4 +108,30 @@ public class GoogleSheetsRootSchema extends AbstractSchema {
   public String getTypeName() {
     return GoogleSheetsStoragePluginConfig.NAME;
   }
+
+  public Map<String, String> getTokenMap() {
+    return this.tokenMap;
+  }
+
+  /**
+   * Drill automatically converts the file token to lower case during DDL queries. Since Google is case-sensitive,
+   * this method insures that we are using the correct file token.
+   * @return A file token in the correct case.
+   */
+  public static String getFileTokenWithCorrectCase(Map<String, String> tokenMap, String lowercaseToken) {
+    if (StringUtils.isMixedCase(lowercaseToken)) {
+      return lowercaseToken;
+    }
+    Set<String> tokens = tokenMap.keySet();
+
+    for (String token : tokens) {
+      if (token.toLowerCase().contentEquals(lowercaseToken)) {
+        return token;
+      }
+    }
+
+    throw UserException.internalError()
+        .message("Could not find token: " + lowercaseToken)
+        .build(logger);
+  }
 }
diff --git a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/utils/GoogleSheetsUtils.java b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/utils/GoogleSheetsUtils.java
index 971c149c73..e3cc597535 100644
--- a/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/utils/GoogleSheetsUtils.java
+++ b/contrib/storage-googlesheets/src/main/java/org/apache/drill/exec/store/googlesheets/utils/GoogleSheetsUtils.java
@@ -35,7 +35,9 @@ import com.google.api.services.sheets.v4.Sheets;
 import com.google.api.services.sheets.v4.Sheets.Spreadsheets.Values.BatchGet;
 import com.google.api.services.sheets.v4.SheetsScopes;
 import com.google.api.services.sheets.v4.model.AddSheetRequest;
+import com.google.api.services.sheets.v4.model.AppendValuesResponse;
 import com.google.api.services.sheets.v4.model.BatchUpdateSpreadsheetRequest;
+import com.google.api.services.sheets.v4.model.DeleteSheetRequest;
 import com.google.api.services.sheets.v4.model.Request;
 import com.google.api.services.sheets.v4.model.Sheet;
 import com.google.api.services.sheets.v4.model.SheetProperties;
@@ -266,7 +268,7 @@ public class GoogleSheetsUtils {
   /**
    * Google Sheets tokens are strings of length 44 that contain upper and lower case letters, numbers and underscores.
    * This function will attempt to identify file tokens.
-   *
+   * <p>
    * Given that Google's spec for file IDs is not officially published, and can change at any time, we will keep the
    * validation as light as possible to prevent future issues, in the event Google changes their file Id structure.
    * @param id A {@link String} containing an unknown identifier
@@ -370,6 +372,25 @@ public class GoogleSheetsUtils {
     return service.spreadsheets().values().get(sheetID, range).execute().getValues();
   }
 
+  /**
+   * Finds a {@link Sheet} from a list of tabs with a given title.  If the sheet is not present,
+   * the function will throw a User Exception.
+   * @param tabName The name of the desired sheet.
+   * @param tabList A {@link List} of {@link Sheet} objects
+   * @return The desired Sheet.
+   */
+  public static Sheet getSheetFromTabList(String tabName, List<Sheet> tabList) {
+    for (Sheet sheet : tabList) {
+      if (sheet.getProperties().getTitle().contentEquals(tabName)) {
+        return sheet;
+      }
+    }
+
+    throw UserException.dataReadError()
+        .message("Could not find sheet " + tabName)
+        .build(logger);
+  }
+
   /**
    * This function is used to get data when projection is pushed down to Google Sheets.
    * @param service The Authenticated GoogleSheets service
@@ -591,6 +612,24 @@ public class GoogleSheetsUtils {
     service.spreadsheets().batchUpdate(sheetName, body).execute();
   }
 
+  /**
+   * Removes a sheet from an existing GoogleSheets document.  This method should only be used if the GoogleSheets
+   * document has more than one tab.
+   * @param service An authenticated GoogleSheet {@link Sheets}
+   * @param fileToken The File token of the GoogleSheet containing the sheet to be deleted
+   * @param deletedTab  A {@link Sheet} which will be removed
+   * @throws IOException If anything goes wrong.
+   */
+  public static void removeTabFromGoogleSheet(Sheets service, String fileToken, Sheet deletedTab) throws IOException {
+    List<Request> requests = new ArrayList<>();
+    requests.add(new Request()
+        .setDeleteSheet(new DeleteSheetRequest().setSheetId(deletedTab.getProperties().getSheetId()))
+    );
+
+    BatchUpdateSpreadsheetRequest body = new BatchUpdateSpreadsheetRequest().setRequests(requests);
+    service.spreadsheets().batchUpdate(fileToken, body).execute();
+  }
+
   /**
    * Accepts a list of data and writes this data to a GoogleSheet document.
    * @param service An authenticated GoogleSheet service
@@ -610,4 +649,24 @@ public class GoogleSheetsUtils {
         .setValueInputOption("RAW")
         .execute();
   }
+
+  /**
+   * Accepts a list of data and writes this data to a GoogleSheet document.
+   * @param service An authenticated GoogleSheet service
+   * @param sheetID The SheetID.  This can be obtained from the URL of the GoogleSheet Document
+   * @param tabName The tab name within the aforementioned GoogleSheet
+   * @param data A list of rows of the data to be inserted.
+   * @throws IOException If anything goes wrong, throw an IO exception
+   */
+  public static void appendDataToGoogleSheet(Sheets service, String sheetID, String tabName, List<List<Object>> data)
+      throws IOException {
+    String range = tabName + "!A1";
+    ValueRange body = new ValueRange()
+        .setValues(data)
+        .setMajorDimension("ROWS");
+
+    AppendValuesResponse result = service.spreadsheets().values().append(sheetID, range, body)
+        .setValueInputOption("RAW")
+        .execute();
+  }
 }
diff --git a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java
index daaba71ebf..2b06de1e85 100644
--- a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java
+++ b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java
@@ -20,8 +20,12 @@ package org.apache.drill.exec.store.googlesheets;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.categories.RowSetTest;
+import org.apache.drill.common.types.TypeProtos.MinorType;
 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.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 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;
@@ -29,6 +33,7 @@ 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.apache.drill.test.rowSet.RowSetComparison;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -40,6 +45,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -54,7 +60,9 @@ public class TestGoogleSheetsWriter extends ClusterTest {
   private static String accessToken;
   private static String refreshToken;
 
-  // Note on testing:  Testing the writing capabilites of this plugin is challenging.
+  private static String sheetID;
+
+  // Note on testing:  Testing the writing capabilities of this plugin is challenging.
   // The primary issue is that when you execute a CTAS query, you do so using the file name.
   // However, it does not seem possible to retrieve the created file's ID which is what you
   // need to actually verify that the query successfully wrote the results.  Therefore, at this
@@ -74,6 +82,7 @@ public class TestGoogleSheetsWriter extends ClusterTest {
     String clientSecret = tokenMap.get("client_secret");
     accessToken = tokenMap.get("access_token");
     refreshToken = tokenMap.get("refresh_token");
+    sheetID = tokenMap.get("sheet_id");
 
     pluginRegistry = cluster.drillbit().getContext().getStorage();
     GoogleSheetsStoragePluginConfig config = GoogleSheetsStoragePluginConfig.builder()
@@ -82,6 +91,8 @@ public class TestGoogleSheetsWriter extends ClusterTest {
       .redirectUris(REDIRECT_URI)
       .authUri(AUTH_URI)
       .tokenUri(TOKEN_URI)
+      .extractHeaders(true)
+      .allTextMode(false)
       .build();
 
     config.setEnabled(true);
@@ -103,6 +114,65 @@ public class TestGoogleSheetsWriter extends ClusterTest {
     assertTrue(insertResults.succeeded());
   }
 
+  @Test
+  public void testCTASLifecycle() throws Exception {
+    // This test goes through the entire CTAS, INSERT, DROP lifecycle.
+    try {
+      initializeTokens();
+    } catch (PluginException e) {
+      fail(e.getMessage());
+    }
+
+    // We are creating a new tab in an existing GS document
+    String sql = String.format("CREATE TABLE googlesheets.`%s`.`Sheet3` AS SELECT * FROM cp.`data/insert_data.csvh`", sheetID);
+    QuerySummary results = queryBuilder().sql(sql).run();
+    assertTrue(results.succeeded());
+
+    // Verify the sheet was created
+    sql = String.format("SELECT * FROM googlesheets.`%s`.`Sheet3`", sheetID);
+    results = queryBuilder().sql(sql).run();
+    assertTrue(results.succeeded());
+    assertEquals(2, results.recordCount());
+
+    // Now Insert additional records into the sheet
+    sql = String.format("INSERT INTO googlesheets.`%s`.`Sheet3` SELECT * FROM cp.`data/insert_data2.csvh`", sheetID);
+    results = queryBuilder().sql(sql).run();
+    assertTrue(results.succeeded());
+
+    // Verify that the records were inserted
+    sql = String.format("SELECT * FROM googlesheets.`%s`.`Sheet3`", sheetID);
+    RowSet rowSet = queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("col1", MinorType.FLOAT8)
+        .addNullable("col2", MinorType.FLOAT8)
+        .addNullable("col3", MinorType.FLOAT8)
+        .buildSchema();
+
+    RowSet expected = client.rowSetBuilder(expectedSchema)
+        .addRow(1,2,3)
+        .addRow(4,5,6)
+        .addRow(7,8,9)
+        .addRow(10,11,12)
+        .build();
+    new RowSetComparison(expected).verifyAndClearAll(rowSet);
+
+    // Drop the table
+    sql = String.format("DROP TABLE googlesheets.`%s`.`Sheet3`", sheetID);
+    results = queryBuilder().sql(sql).run();
+    assertTrue(results.succeeded());
+
+    // Verify that it's gone
+    sql = String.format("SELECT * FROM googlesheets.`%s`.`Sheet3`", sheetID);
+    try {
+      results = queryBuilder().sql(sql).run();
+      fail();
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("'Sheet3' not found"));
+    }
+ }
+
+
   /**
    * This function is used for testing only.  It initializes a {@link PersistentTokenTable} and populates it
    * with a valid access and refresh token.
diff --git a/contrib/storage-googlesheets/src/test/resources/data/insert_data.csvh b/contrib/storage-googlesheets/src/test/resources/data/insert_data.csvh
new file mode 100644
index 0000000000..e39967f2f0
--- /dev/null
+++ b/contrib/storage-googlesheets/src/test/resources/data/insert_data.csvh
@@ -0,0 +1,3 @@
+col1, col2, col3
+1,2,3
+4,5,6
\ No newline at end of file
diff --git a/contrib/storage-googlesheets/src/test/resources/data/insert_data2.csvh b/contrib/storage-googlesheets/src/test/resources/data/insert_data2.csvh
new file mode 100644
index 0000000000..7128ece2e8
--- /dev/null
+++ b/contrib/storage-googlesheets/src/test/resources/data/insert_data2.csvh
@@ -0,0 +1,3 @@
+col1, col2, col3
+7,8,9
+10,11,12
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilities.java
index 780270eaca..5afbf06f2a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilities.java
@@ -96,6 +96,7 @@ public class SchemaUtilities {
   /** Utility method to search for schema path starting from the given <i>schema</i> reference */
   public static SchemaPlus searchSchemaTree(SchemaPlus schema, final List<String> schemaPath) {
     for (String schemaName : schemaPath) {
+
       // schemas in Drill are case insensitive and stored in lower case
       schema = schema.getSubSchema(schemaName.toLowerCase());
       if (schema == null) {