You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/11/30 14:06:02 UTC

carbondata git commit: [CARBONDATA-3140]Block create like table command in carbon

Repository: carbondata
Updated Branches:
  refs/heads/master f52f97ea6 -> 4554d191b


[CARBONDATA-3140]Block create like table command in carbon

Why this PR?
when like table is created using carbon table as source table, and the new table is dropped, it deletes the source table in spark-2.1 and works fine in other. Blocking the create like command for carbon table

This closes #2967


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/4554d191
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/4554d191
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/4554d191

Branch: refs/heads/master
Commit: 4554d191b8a545b5afe4e15d19d69515a56eaad1
Parents: f52f97e
Author: akashrn5 <ak...@gamil.com>
Authored: Fri Nov 30 16:01:34 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Nov 30 19:35:53 2018 +0530

----------------------------------------------------------------------
 .../carbondata/core/memory/UnsafeMemoryManager.java   |  2 +-
 .../createTable/TestCreateTableIfNotExists.scala      | 14 ++++++++++++++
 .../spark/sql/execution/strategy/DDLStrategy.scala    |  9 +++++++++
 3 files changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/4554d191/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
index 493ec0b..c59698f 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
@@ -129,7 +129,7 @@ public class UnsafeMemoryManager {
       memoryBlock = MemoryAllocator.HEAP.allocate(memoryRequested);
       if (LOGGER.isDebugEnabled()) {
         LOGGER.debug(String
-            .format("Creating onheap working Memory block (%d) with size: ", memoryBlock.size()));
+            .format("Creating onheap working Memory block with size: (%d)", memoryBlock.size()));
       }
     }
     return memoryBlock;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4554d191/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableIfNotExists.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableIfNotExists.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableIfNotExists.scala
index a8fcf12..8f7afe4 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableIfNotExists.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableIfNotExists.scala
@@ -20,11 +20,15 @@ package org.apache.carbondata.spark.testsuite.createTable
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+
 class TestCreateTableIfNotExists extends QueryTest with BeforeAndAfterAll {
 
   override def beforeAll {
     sql("use default")
     sql("drop table if exists test")
+    sql("drop table if exists sourceTable")
+    sql("drop table if exists targetTable")
   }
 
   test("test create table if not exists") {
@@ -39,9 +43,19 @@ class TestCreateTableIfNotExists extends QueryTest with BeforeAndAfterAll {
     }
   }
 
+  test("test blocking of create table like command") {
+    sql("create table sourceTable(name string) stored by 'carbondata'")
+    val exception = intercept[MalformedCarbonCommandException] {
+      sql("create table targetTable like sourceTable")
+    }
+    assert(exception.getMessage.contains("Operation not allowed, when source table is carbon table"))
+  }
+
   override def afterAll {
     sql("use default")
     sql("drop table if exists test")
+    sql("drop table if exists sourceTable")
+    sql("drop table if exists targetTable")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4554d191/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index 6e183b2..4cc62c6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -92,6 +92,15 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
         ExecutedCommandExec(
           CarbonDropTableCommand(ifNotExists, identifier.database,
             identifier.table.toLowerCase)) :: Nil
+      case createLikeTable: CreateTableLikeCommand =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(createLikeTable.sourceTable)(sparkSession)
+        if (isCarbonTable) {
+          throw new MalformedCarbonCommandException(
+            "Operation not allowed, when source table is carbon table")
+        } else {
+          ExecutedCommandExec(createLikeTable) :: Nil
+        }
       case InsertIntoCarbonTable(relation: CarbonDatasourceHadoopRelation,
       partition, child: LogicalPlan, overwrite, _) =>
         ExecutedCommandExec(CarbonInsertIntoCommand(relation, child, overwrite, partition)) :: Nil