You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by in...@apache.org on 2020/12/16 06:56:20 UTC

[carbondata] branch master updated: [CARBONDATA-4069] [CARBONDATA-4068] handled set streaming and long string for SI table or table having SI

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

indhumuthumurugesh 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 6b03967  [CARBONDATA-4069] [CARBONDATA-4068] handled set streaming and long string for SI table or table having SI
6b03967 is described below

commit 6b0396774ac46db7264d71874f5fe315fad4fead
Author: Nihal ojha <ni...@gmail.com>
AuthorDate: Fri Dec 4 12:34:43 2020 +0530

    [CARBONDATA-4069] [CARBONDATA-4068] handled set streaming and long string for SI table or table having SI
    
    Why is this PR needed?
    Currently alter table set streaming = true is allowed either for SI table or MT which has SI.
    Currently set column as long string on which SI is already created is allowed. This operation
    should not be allowed because SI doesn't support long strings.
    
    What changes were proposed in this PR?
    Handled set streaming true for SI table and MT which has SI and throws the exception.
    Handled set long string on SI columns and throws the exception.
    
    Does this PR introduce any user interface change?
    No
    
    Is any new testcase added?
    Yes
    
    This closes #4042
---
 .../secondaryindex/TestSIWithSecondaryIndex.scala  | 32 ++++++++++++++++++++++
 .../org/apache/spark/util/AlterTableUtil.scala     | 32 ++++++++++++++++++++--
 2 files changed, 61 insertions(+), 3 deletions(-)

diff --git a/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala b/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala
index 286dea5..f424d91 100644
--- a/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala
+++ b/index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithSecondaryIndex.scala
@@ -434,6 +434,38 @@ class TestSIWithSecondaryIndex extends QueryTest with BeforeAndAfterAll {
     assert(df.exists(_.get(0).toString.contains("nam(e")))
   }
 
+  test("test alter table set streaming for table having SI") {
+    sql("drop table if exists maintable2")
+    sql("create table maintable2 (a string,b string,c int) STORED AS carbondata ")
+    sql("insert into maintable2 values('k','x',2)")
+    sql("create index m_indextable on table maintable2(b) AS 'carbondata'")
+    var exeption = intercept[RuntimeException] {
+      sql("ALTER TABLE maintable2 SET TBLPROPERTIES('streaming'='true')")
+    }
+    assert(exeption.getMessage.contains("Set streaming table is " +
+      "not allowed for tables which are having index(s)."))
+
+    exeption = intercept[RuntimeException] {
+      sql("ALTER TABLE m_indextable SET TBLPROPERTIES('streaming'='true')")
+    }
+    assert(exeption.getMessage.contains("Set streaming table is not allowed on the index table."))
+    sql("drop table if exists maintable2")
+  }
+
+  test("test change data type from string to long string of SI column") {
+    sql("drop table if exists maintable")
+    sql("create table maintable (a string,b string,c int) STORED AS carbondata ")
+    sql("create index indextable on table maintable(b) AS 'carbondata'")
+    sql("insert into maintable values('k','x',2)")
+    val exception = intercept[RuntimeException] {
+      sql("ALTER TABLE maintable SET TBLPROPERTIES('long_String_columns'='b')")
+    }
+    assert(exception.getMessage.contains("Cannot Alter column b to " +
+      "Long_string_column, as the column exists in a secondary index with name " +
+      "indextable. LONG_STRING_COLUMNS is not allowed on secondary index."))
+    sql("drop table if exists maintable")
+  }
+
   override def afterAll {
     dropIndexAndTable()
   }
diff --git a/integration/spark/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 07b7dba..c6eca1c 100644
--- a/integration/spark/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -24,12 +24,11 @@ import scala.collection.mutable
 import scala.collection.mutable.ListBuffer
 
 import org.apache.commons.lang3.StringUtils
-import org.apache.spark.SparkConf
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.SessionCatalog
 import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionCatalogUtil}
-import org.apache.spark.sql.hive.HiveExternalCatalog._
+import org.apache.spark.sql.index.CarbonIndexUtil
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -41,8 +40,9 @@ import org.apache.carbondata.core.exception.InvalidConfigurationException
 import org.apache.carbondata.core.index.IndexStoreManager
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.converter.{SchemaConverter, ThriftWrapperSchemaConverterImpl}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.index.IndexType
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
@@ -458,6 +458,19 @@ object AlterTableUtil {
           updateSchemaForLongStringColumns(thriftTable, "")
         }
       }
+      // validate set for streaming table
+      val streamingOption = lowerCasePropertiesMap.get("streaming")
+      if (streamingOption.isDefined && set) {
+        if (carbonTable.isIndexTable) {
+          throw new UnsupportedOperationException("Set streaming table is " +
+            "not allowed on the index table.")
+        }
+        val indexTables = CarbonIndexUtil.getSecondaryIndexes(carbonTable)
+        if (!indexTables.isEmpty) {
+          throw new UnsupportedOperationException("Set streaming table is " +
+            "not allowed for tables which are having index(s).")
+        }
+      }
       // below map will be used for cache invalidation. As tblProperties map is getting modified
       // in the next few steps the original map need to be retained for any decision making
       val existingTablePropertiesMap = mutable.Map(tblPropertiesMap.toSeq: _*)
@@ -916,6 +929,19 @@ object AlterTableUtil {
         }
       }
     }
+    // should not be present in index tables
+    val secondaryIndexMap = carbonTable.getIndexesMap.get(IndexType.SI.getIndexProviderName)
+    if (secondaryIndexMap != null) {
+      secondaryIndexMap.asScala.foreach(indexTable => {
+        indexTable._2.asScala(CarbonCommonConstants.INDEX_COLUMNS).split(",").foreach(col =>
+          if (longStringCols.contains(col.toLowerCase)) {
+            throw new MalformedCarbonCommandException(s"Cannot Alter column $col to " +
+              s"Long_string_column, as the column exists in a secondary index with name " +
+              s"${indexTable._1}. LONG_STRING_COLUMNS is not allowed on secondary index.")
+          }
+        )
+      })
+    }
   }
 
   /**