You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ma...@apache.org on 2017/11/30 15:39:40 UTC

carbondata git commit: [CARBONDATA-1802][Alter] Alter query fails if a column is dropped and there is no key column

Repository: carbondata
Updated Branches:
  refs/heads/master a31598ea5 -> 3a3e85f62


[CARBONDATA-1802][Alter] Alter query fails if a column is dropped and there is no key column

1. Alter query fails if a column is dropped and there is no key column. All key columns in a carbon table can be dropped, as we are supporting table creation with no keycolumn, alter also should be consistent with this

2. Table already exists exception is wrong. Corrected error message

This closes #1561


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

Branch: refs/heads/master
Commit: 3a3e85f62eb6b07df75d0f60bddc48e841cecc84
Parents: a31598e
Author: dhatchayani <dh...@gmail.com>
Authored: Fri Nov 24 14:23:04 2017 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Thu Nov 30 21:12:34 2017 +0530

----------------------------------------------------------------------
 .../schema/CarbonAlterTableDropColumnCommand.scala   | 10 ----------
 .../command/table/CarbonCreateTableCommand.scala     |  2 +-
 .../restructure/AlterTableValidationTestCase.scala   | 15 +++++++++++++++
 3 files changed, 16 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3a3e85f6/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index 77efc10..c5924b6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -69,7 +69,6 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
       val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
       var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
       .ColumnSchema]()
-      var keyColumnCountToBeDeleted = 0
       // TODO: if deleted column list includes bucketted column throw an error
       alterTableDropColumnModel.columns.foreach { column =>
         var columnExist = false
@@ -77,7 +76,6 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
           // column should not be already deleted and should exist in the table
           if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
             if (tableColumn.isDimension) {
-              keyColumnCountToBeDeleted += 1
               if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
                 dictionaryColumns ++= Seq(tableColumn.getColumnSchema)
               }
@@ -89,14 +87,6 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
           sys.error(s"Column $column does not exists in the table $dbName.$tableName")
         }
       }
-      // take the total key column count. key column to be deleted should not
-      // be >= key columns in schema
-      val totalKeyColumnInSchema = tableColumns.count {
-        tableColumn => !tableColumn.isInvisible && tableColumn.isDimension
-      }
-      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
-        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
-      }
 
       val operationContext = new OperationContext
       // event will be fired before dropping the columns

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3a3e85f6/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
index 33d9924..9198f57 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateTableCommand.scala
@@ -71,7 +71,7 @@ case class CarbonCreateTableCommand(
         LOGGER.audit(
           s"Table creation with Database name [$dbName] and Table name [$tbName] failed. " +
           s"Table [$tbName] already exists under database [$dbName]")
-        throw new TableAlreadyExistsException(dbName, dbName)
+        throw new TableAlreadyExistsException(dbName, tbName)
       }
     } else {
       val tableIdentifier = AbsoluteTableIdentifier.from(tablePath, dbName, tbName)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3a3e85f6/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
index 48cf1be..4dc3ee3 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
@@ -44,6 +44,7 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
     sql("drop table if exists restructure_new")
     sql("drop table if exists restructure_bad")
     sql("drop table if exists restructure_badnew")
+    sql("drop table if exists allKeyCol")
     // clean data folder
     CarbonProperties.getInstance()
     sql(
@@ -151,6 +152,19 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
     checkExistence(sql("desc restructure"), true, "dcmldecimal(5,4)")
   }
 
+  test("test drop all keycolumns in a table") {
+    sql(
+      "create table allKeyCol (name string, age int, address string) stored by 'org.apache" +
+      ".carbondata.format'")
+    try {
+      sql("alter table allKeyCol drop columns(name,address)")
+      assert(true)
+    } catch {
+      case _: Exception =>
+        assert(false)
+    }
+  }
+
   ignore(
     "test add decimal without scale and precision, default precision and scale (10,0) should be " +
     "used")
@@ -526,5 +540,6 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
     sql("drop table if exists uniqdata1")
     sql("drop table if exists defaultSortColumnsWithAlter")
     sql("drop table if exists specifiedSortColumnsWithAlter")
+    sql("drop table if exists allKeyCol")
   }
 }