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

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


The following commit(s) were added to refs/heads/master by this push:
     new 78b20a8  [CARBONDATA-3696] Avoid list db's all Tables to check table 
exists in the db
78b20a8 is described below

commit 78b20a8a7ac8223bafb9f304dd28be44eb1900c3
Author: ShuMingLi <ming.moria...@gmail.com>
AuthorDate: Thu Feb 13 10:37:49 2020 +0800

    [CARBONDATA-3696] Avoid list db's all Tables to check table exists in the db
    
    Why is this PR needed?
    
    I found a lot of listTables of the db to check if table exists like this:
    
    sparkSession.sessionState.catalog.listTables(databaseName)
      .exists(_.table.equalsIgnoreCase(tableName)
    This may affect performances if there are lots of tables in one database. 
In spark, we can check this by tableExists function.
    
    What changes were proposed in this PR?
    use exits(db, table) to replace iterate tables in a db.
    
    Does this PR introduce any user interface change?
    No
    
    Is any new testcase added?
    No
    
    This closes #3631
---
 .../execution/command/table/CarbonCreateTableAsSelectCommand.scala    | 4 ++--
 .../spark/sql/execution/command/table/CarbonCreateTableCommand.scala  | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git 
a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableAsSelectCommand.scala
 
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableAsSelectCommand.scala
index 9725b4e..751d2f4 100644
--- 
a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableAsSelectCommand.scala
+++ 
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableAsSelectCommand.scala
@@ -54,8 +54,8 @@ case class CarbonCreateTableAsSelectCommand(
     setAuditTable(dbName, tableName)
     setAuditInfo(Map("query" -> query.simpleString))
     // check if table already exists
-    if (sparkSession.sessionState.catalog.listTables(dbName)
-      .exists(_.table.equalsIgnoreCase(tableName))) {
+    if (sparkSession.sessionState.catalog
+      .tableExists(TableIdentifier(tableName, Some(dbName)))) {
       if (!ifNotExistsSet) {
         throw new TableAlreadyExistsException(dbName, tableName)
       }
diff --git 
a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
 
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
index 7d9e3ce..01d2693 100644
--- 
a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
+++ 
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
@@ -62,8 +62,8 @@ case class CarbonCreateTableCommand(
     tableInfo.setDatabaseName(dbName)
     tableInfo.setTableUniqueName(CarbonTable.buildUniqueName(dbName, 
tableName))
     val isTransactionalTable = tableInfo.isTransactionalTable
-    if (sparkSession.sessionState.catalog.listTables(dbName)
-      .exists(_.table.equalsIgnoreCase(tableName))) {
+    if (sparkSession.sessionState.catalog
+      .tableExists(TableIdentifier(tableName, Some(dbName)))) {
       if (!ifNotExistsSet) {
         throw new TableAlreadyExistsException(dbName, tableName)
       }

Reply via email to