carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ravipes...@apache.org
Subject [1/2] carbondata git commit: [CARBONDATA-1061] if AL_DICTIONARY_PATH is used in load option then by SINGLE_PASS must be used.
Date Wed, 17 May 2017 19:36:39 GMT
Repository: carbondata
Updated Branches:
  refs/heads/master 07b6e9f3a -> 608fb3a26


[CARBONDATA-1061] if AL_DICTIONARY_PATH is used in load option then by SINGLE_PASS must be
used.


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

Branch: refs/heads/master
Commit: 083a28a8c56bfe3f8ac7c532562fc91108745517
Parents: 07b6e9f
Author: mohammadshahidkhan <mohdshahidkhan1987@gmail.com>
Authored: Wed May 17 19:35:54 2017 +0530
Committer: ravipesala <ravi.pesala@gmail.com>
Committed: Thu May 18 01:05:37 2017 +0530

----------------------------------------------------------------------
 .../resources/predefdic/allpredefdictionary.csv |   3 +
 .../src/test/resources/predefdic/data3.csv      |   4 +
 .../test/resources/predefdic/dicfilepath.csv    |   2 +
 .../predefdic/TestPreDefDictionary.scala        |  97 +++++++
 .../spark/util/GlobalDictionaryUtil.scala       |  78 +++--
 .../execution/command/carbonTableSchema.scala   |  45 +--
 .../execution/command/carbonTableSchema.scala   |  46 +--
 .../spark/util/AllDictionaryTestCase.scala      | 160 +++++++++++
 .../spark/util/DictionaryTestCaseUtil.scala     |  49 ++++
 .../util/ExternalColumnDictionaryTestCase.scala | 284 +++++++++++++++++++
 10 files changed, 699 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark-common-test/src/test/resources/predefdic/allpredefdictionary.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/predefdic/allpredefdictionary.csv
b/integration/spark-common-test/src/test/resources/predefdic/allpredefdictionary.csv
new file mode 100644
index 0000000..27152ca
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/predefdic/allpredefdictionary.csv
@@ -0,0 +1,3 @@
+1,phone756
+1,phonetype
+1,phone757

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark-common-test/src/test/resources/predefdic/data3.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/predefdic/data3.csv b/integration/spark-common-test/src/test/resources/predefdic/data3.csv
new file mode 100644
index 0000000..c84506a
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/predefdic/data3.csv
@@ -0,0 +1,4 @@
+ID,phonetype
+1,phone197
+2,phone756
+3,phone757
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark-common-test/src/test/resources/predefdic/dicfilepath.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/predefdic/dicfilepath.csv b/integration/spark-common-test/src/test/resources/predefdic/dicfilepath.csv
new file mode 100644
index 0000000..e49a020
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/predefdic/dicfilepath.csv
@@ -0,0 +1,2 @@
+phone756
+phone757

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/predefdic/TestPreDefDictionary.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/predefdic/TestPreDefDictionary.scala
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/predefdic/TestPreDefDictionary.scala
new file mode 100644
index 0000000..69af708
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/predefdic/TestPreDefDictionary.scala
@@ -0,0 +1,97 @@
+/*
+ * 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.carbondata.spark.testsuite.predefdic
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * Test cases for testing columns having \N or \null values for non numeric columns
+ */
+class TestPreDefDictionary extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("DROP TABLE IF EXISTS predefdictable")
+    sql("DROP TABLE IF EXISTS predefdictable1")
+    sql("DROP TABLE IF EXISTS columndicTable")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT
+      )
+  }
+
+  test("dictionary value not present in the allpredefdictionary dic file must be loaded.")
{
+    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
+    val testData = s"$resourcesPath/predefdic/data3.csv"
+    val csvHeader = "ID,phonetype"
+    val allDictFile = s"$resourcesPath/predefdic/allpredefdictionary.csv"
+    sql(
+      """CREATE TABLE IF NOT EXISTS predefdictable (ID Int, phonetype String)
+       STORED BY 'carbondata'""")
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$testData' into table predefdictable
+           options('ALL_DICTIONARY_PATH'='$allDictFile')""")
+    checkAnswer(
+      sql("select phonetype from predefdictable where phonetype='phone197'"),
+      Seq(Row("phone197"))
+    )
+  }
+
+  test("dictionary value not present in the allpredefdictionary dic with single_pass.") {
+    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
+    val testData = s"$resourcesPath/predefdic/data3.csv"
+    val csvHeader = "ID,phonetype"
+    val allDictFile = s"$resourcesPath/predefdic/allpredefdictionary.csv"
+    sql(
+      """CREATE TABLE IF NOT EXISTS predefdictable1 (ID Int, phonetype String)
+       STORED BY 'carbondata'""")
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$testData' into table predefdictable1
+           options('ALL_DICTIONARY_PATH'='$allDictFile', 'SINGLE_PASS'='true')""")
+    checkAnswer(
+      sql("select phonetype from predefdictable1 where phonetype='phone197'"),
+      Seq(Row("phone197"))
+    )
+  }
+
+  test("dictionary value not present in the columndict dic with single_pass.") {
+    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
+    val testData = s"$resourcesPath/predefdic/data3.csv"
+    val csvHeader = "ID,phonetype"
+    val dicFilePath = s"$resourcesPath/predefdic/dicfilepath.csv"
+    sql(
+      """CREATE TABLE IF NOT EXISTS columndicTable (ID Int, phonetype String)
+       STORED BY 'carbondata'""")
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$testData' into table columndicTable
+           options('COLUMNDICT'='phonetype:$dicFilePath', 'SINGLE_PASS'='true')""")
+    checkAnswer(
+      sql("select phonetype from columndicTable where phonetype='phone197'"),
+      Seq(Row("phone197"))
+    )
+  }
+  override def afterAll {
+    sql("DROP TABLE IF EXISTS predefdictable")
+    sql("DROP TABLE IF EXISTS predefdictable1")
+    sql("DROP TABLE IF EXISTS columndicTable")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index f690eef..549bdf9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -750,37 +750,13 @@ object GlobalDictionaryUtil {
           LOGGER.info("No column found for generating global dictionary in source data files")
         }
       } else {
-        LOGGER.info("Generate global dictionary from dictionary files!")
-        val isNonempty = validateAllDictionaryPath(allDictionaryPath)
-        if (isNonempty) {
-          var headers = carbonLoadModel.getCsvHeaderColumns
-          headers = headers.map(headerName => headerName.trim)
-          // prune columns according to the CSV file header, dimension columns
-          val (requireDimension, requireColumnNames) = pruneDimensions(dimensions, headers,
headers)
-          if (requireDimension.nonEmpty) {
-            val model = createDictionaryLoadModel(carbonLoadModel, carbonTableIdentifier,
-              requireDimension, storePath, dictfolderPath, false)
-            // check if dictionary files contains bad record
-            val accumulator = sqlContext.sparkContext.accumulator(0)
-            // read local dictionary file, and group by key
-            val allDictionaryRdd = readAllDictionaryFiles(sqlContext, headers,
-              requireColumnNames, allDictionaryPath, accumulator)
-            // read exist dictionary and combine
-            val inputRDD = new CarbonAllDictionaryCombineRDD(allDictionaryRdd, model)
-              .partitionBy(new ColumnPartitioner(model.primDimensions.length))
-            // generate global dictionary files
-            val statusList = new CarbonGlobalDictionaryGenerateRDD(inputRDD, model).collect()
-            // check result status
-            checkStatus(carbonLoadModel, sqlContext, model, statusList)
-            // if the dictionary contains wrong format record, throw ex
-            if (accumulator.value > 0) {
-              throw new DataLoadingException("Data Loading failure, dictionary values are
" +
-                                             "not in correct format!")
-            }
-          } else {
-            LOGGER.info("have no column need to generate global dictionary")
-          }
-        }
+        generateDictionaryFromDictionaryFiles(sqlContext,
+          carbonLoadModel,
+          storePath,
+          carbonTableIdentifier,
+          dictfolderPath,
+          dimensions,
+          allDictionaryPath)
       }
     } catch {
       case ex: Exception =>
@@ -796,6 +772,46 @@ object GlobalDictionaryUtil {
     }
   }
 
+  def generateDictionaryFromDictionaryFiles(sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      storePath: String,
+      carbonTableIdentifier: CarbonTableIdentifier,
+      dictfolderPath: String,
+      dimensions: Array[CarbonDimension],
+      allDictionaryPath: String): Unit = {
+    LOGGER.info("Generate global dictionary from dictionary files!")
+    val isNonempty = validateAllDictionaryPath(allDictionaryPath)
+    if (isNonempty) {
+      var headers = carbonLoadModel.getCsvHeaderColumns
+      headers = headers.map(headerName => headerName.trim)
+      // prune columns according to the CSV file header, dimension columns
+      val (requireDimension, requireColumnNames) = pruneDimensions(dimensions, headers, headers)
+      if (requireDimension.nonEmpty) {
+        val model = createDictionaryLoadModel(carbonLoadModel, carbonTableIdentifier,
+          requireDimension, storePath, dictfolderPath, false)
+        // check if dictionary files contains bad record
+        val accumulator = sqlContext.sparkContext.accumulator(0)
+        // read local dictionary file, and group by key
+        val allDictionaryRdd = readAllDictionaryFiles(sqlContext, headers,
+          requireColumnNames, allDictionaryPath, accumulator)
+        // read exist dictionary and combine
+        val inputRDD = new CarbonAllDictionaryCombineRDD(allDictionaryRdd, model)
+          .partitionBy(new ColumnPartitioner(model.primDimensions.length))
+        // generate global dictionary files
+        val statusList = new CarbonGlobalDictionaryGenerateRDD(inputRDD, model).collect()
+        // check result status
+        checkStatus(carbonLoadModel, sqlContext, model, statusList)
+        // if the dictionary contains wrong format record, throw ex
+        if (accumulator.value > 0) {
+          throw new DataLoadingException("Data Loading failure, dictionary values are " +
+                                         "not in correct format!")
+        }
+      } else {
+        LOGGER.info("have no column need to generate global dictionary")
+      }
+    }
+  }
+
   // Get proper error message of TextParsingException
   def trimErrorMessage(input: String): String = {
     var errorMessage: String = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 9745ddd..ac51fa0 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -53,7 +53,7 @@ import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadM
 import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DataManagementFunc, DictionaryLoadModel}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, GlobalDictionaryUtil}
+import org.apache.carbondata.spark.util.{CommonUtil, GlobalDictionaryUtil}
 
 object Checker {
   def validateTableExists(
@@ -425,18 +425,16 @@ case class LoadTable(
       // when single_pass=true, and not use all dict
       val useOnePass = options.getOrElse("single_pass", "false").trim.toLowerCase match {
         case "true" =>
-          if (StringUtils.isEmpty(allDictionaryPath)) {
+          true
+        case "false" =>
+          if (!StringUtils.isEmpty(allDictionaryPath)) {
             true
           } else {
-            LOGGER.error("Can't use single_pass, because SINGLE_PASS and ALL_DICTIONARY_PATH"
+
-              "can not be used together")
             false
           }
-        case "false" =>
-          false
         case illegal =>
           LOGGER.error(s"Can't use single_pass, because illegal syntax found: [" + illegal
+ "] " +
-            "Please set it as 'true' or 'false'")
+                       "Please set it as 'true' or 'false'")
           false
       }
       carbonLoadModel.setUseOnePass(useOnePass)
@@ -470,25 +468,36 @@ case class LoadTable(
           maxColumns)
         carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
         GlobalDictionaryUtil.updateTableMetadataFunc = updateTableMetadata
-
+        val storePath = relation.tableMeta.storePath
+        val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+        val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+          .getCarbonTableIdentifier
+        val carbonTablePath = CarbonStorePath
+          .getCarbonTablePath(storePath, carbonTableIdentifier)
+        val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+        val dimensions = carbonTable.getDimensionByTableName(
+          carbonTable.getFactTableName).asScala.toArray
         if (carbonLoadModel.getUseOnePass) {
           val colDictFilePath = carbonLoadModel.getColDictFilePath
-          if (colDictFilePath != null) {
-            val storePath = relation.tableMeta.storePath
-            val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-            val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-              .getCarbonTableIdentifier
-            val carbonTablePath = CarbonStorePath
-              .getCarbonTablePath(storePath, carbonTableIdentifier)
-            val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
-            val dimensions = carbonTable.getDimensionByTableName(
-              carbonTable.getFactTableName).asScala.toArray
+          if (!StringUtils.isEmpty(colDictFilePath)) {
             carbonLoadModel.initPredefDictMap()
             // generate predefined dictionary
             GlobalDictionaryUtil
               .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
                 dimensions, carbonLoadModel, sqlContext, storePath, dictFolderPath)
           }
+          val allDictPath: String = carbonLoadModel.getAllDictPath
+          if(!StringUtils.isEmpty(allDictPath)) {
+            carbonLoadModel.initPredefDictMap()
+            GlobalDictionaryUtil
+              .generateDictionaryFromDictionaryFiles(sqlContext,
+                carbonLoadModel,
+                storePath,
+                carbonTableIdentifier,
+                dictFolderPath,
+                dimensions,
+                allDictionaryPath)
+          }
           // dictionaryServerClient dictionary generator
           val dictionaryServerPort = CarbonProperties.getInstance()
             .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 94a95fd..6bc9e61 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -41,22 +41,19 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
-import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension}
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.processing.constants.TableOptionConstant
 import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CarbonSparkUtil, CommonUtil,
-DataTypeConverterUtil, GlobalDictionaryUtil}
+import org.apache.carbondata.spark.util.{CarbonSparkUtil, CommonUtil, GlobalDictionaryUtil}
 
 object Checker {
   def validateTableExists(
@@ -436,15 +433,13 @@ case class LoadTable(
           DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + isEmptyDataBadRecord)
       val useOnePass = options.getOrElse("single_pass", "false").trim.toLowerCase match {
         case "true" =>
-          if (StringUtils.isEmpty(allDictionaryPath)) {
+          true
+        case "false" =>
+          if (!StringUtils.isEmpty(allDictionaryPath)) {
             true
           } else {
-            LOGGER.error("Can't use single_pass, because SINGLE_PASS and ALL_DICTIONARY_PATH"
+
-                         "can not be used together")
             false
           }
-        case "false" =>
-          false
         case illegal =>
           LOGGER.error(s"Can't use single_pass, because illegal syntax found: [" + illegal
+ "] " +
                        "Please set it as 'true' or 'false'")
@@ -480,23 +475,34 @@ case class LoadTable(
         carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
         GlobalDictionaryUtil.updateTableMetadataFunc = LoadTable.updateTableMetadata
         if (carbonLoadModel.getUseOnePass) {
+          val storePath = relation.tableMeta.storePath
+          val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+          val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+            .getCarbonTableIdentifier
+          val carbonTablePath = CarbonStorePath
+            .getCarbonTablePath(storePath, carbonTableIdentifier)
+          val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+          val dimensions = carbonTable.getDimensionByTableName(
+            carbonTable.getFactTableName).asScala.toArray
           val colDictFilePath = carbonLoadModel.getColDictFilePath
-          if (colDictFilePath != null) {
-            val storePath = relation.tableMeta.storePath
-            val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-            val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-              .getCarbonTableIdentifier
-            val carbonTablePath = CarbonStorePath
-              .getCarbonTablePath(storePath, carbonTableIdentifier)
-            val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
-            val dimensions = carbonTable.getDimensionByTableName(
-              carbonTable.getFactTableName).asScala.toArray
+          if (!StringUtils.isEmpty(colDictFilePath)) {
             carbonLoadModel.initPredefDictMap()
             // generate predefined dictionary
             GlobalDictionaryUtil
               .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
                 dimensions, carbonLoadModel, sparkSession.sqlContext, storePath, dictFolderPath)
           }
+          if (!StringUtils.isEmpty(allDictionaryPath)) {
+            carbonLoadModel.initPredefDictMap()
+            GlobalDictionaryUtil
+              .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
+                carbonLoadModel,
+                storePath,
+                carbonTableIdentifier,
+                dictFolderPath,
+                dimensions,
+                allDictionaryPath)
+          }
           // dictionaryServerClient dictionary generator
           val dictionaryServerPort = CarbonProperties.getInstance()
             .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
new file mode 100644
index 0000000..23800ee
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
@@ -0,0 +1,160 @@
+/*
+ * 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.carbondata.spark.util
+
+import org.apache.spark.sql.common.util.QueryTest
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.constants.TableOptionConstant
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+
+/**
+  * Test Case for org.apache.carbondata.integration.spark.util.GlobalDictionaryUtil
+  */
+class AllDictionaryTestCase extends QueryTest with BeforeAndAfterAll {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  var pwd: String = _
+  var sampleRelation: CarbonRelation = _
+  var complexRelation: CarbonRelation = _
+  var sampleAllDictionaryFile: String = _
+  var complexAllDictionaryFile: String = _
+
+  def buildCarbonLoadModel(relation: CarbonRelation,
+    filePath: String,
+    header: String,
+    allDictFilePath: String): CarbonLoadModel = {
+    val carbonLoadModel = new CarbonLoadModel
+    carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
+    carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    val table = relation.tableMeta.carbonTable
+    val carbonSchema = new CarbonDataLoadSchema(table)
+    carbonLoadModel.setDatabaseName(table.getDatabaseName)
+    carbonLoadModel.setTableName(table.getFactTableName)
+    carbonLoadModel.setCarbonDataLoadSchema(carbonSchema)
+    carbonLoadModel.setFactFilePath(filePath)
+    carbonLoadModel.setCsvHeader(header)
+    carbonLoadModel.setCsvDelimiter(",")
+    carbonLoadModel.setComplexDelimiterLevel1("\\$")
+    carbonLoadModel.setComplexDelimiterLevel2("\\:")
+    carbonLoadModel.setAllDictPath(allDictFilePath)
+    carbonLoadModel.setSerializationNullFormat(
+          TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + ",\\N")
+    carbonLoadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+    carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+    carbonLoadModel
+  }
+
+  override def beforeAll {
+    sql("drop table if exists sample")
+    sql("drop table if exists complextypes")
+    buildTestData
+    // second time comment this line
+    buildTable
+    buildRelation
+  }
+
+  def buildTestData() = {
+    sampleAllDictionaryFile = s"${resourcesPath}/alldictionary/sample/20160423/1400_1405/*.dictionary"
+    complexAllDictionaryFile = s"${resourcesPath}/alldictionary/complex/20160423/1400_1405/*.dictionary"
+  }
+
+  def buildTable() = {
+    try {
+      sql(
+        "CREATE TABLE IF NOT EXISTS sample (id STRING, name STRING, city STRING, " +
+          "age INT) STORED BY 'org.apache.carbondata.format'"
+      )
+    } catch {
+      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
+    }
+    try {
+      sql(
+        "create table complextypes (deviceInformationId string, channelsId string, " +
+          "ROMSize string, purchasedate string, mobile struct<imei: string, imsi: string>,
MAC " +
+          "array<string>, locationinfo array<struct<ActiveAreaId: INT, ActiveCountry:
string, " +
+          "ActiveProvince: string, Activecity: string, ActiveDistrict: string, ActiveStreet:
" +
+          "string>>, proddate struct<productionDate: string,activeDeactivedate:
array<string>>, " +
+          "gamePointId INT,contractNumber INT) STORED BY 'org.apache.carbondata.format'"
+
+          "TBLPROPERTIES('DICTIONARY_EXCLUDE'='ROMSize')"
+      )
+    } catch {
+      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
+    }
+  }
+
+  def buildRelation() = {
+    val warehouse = s"$resourcesPath/target/warehouse"
+    val storeLocation = s"$resourcesPath/target/store"
+    val metastoredb = s"$resourcesPath/target"
+    CarbonProperties.getInstance()
+      .addProperty("carbon.custom.distribution", "true")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,"FORCE")
+    import org.apache.spark.sql.CarbonSession._
+
+    val spark = SparkSession
+      .builder()
+      .master("local")
+      .appName("CarbonSessionExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .config("spark.network.timeout", "600s")
+      .config("spark.executor.heartbeatInterval", "600s")
+      .config("carbon.enable.vector.reader","false")
+      .getOrCreateCarbonSession(storeLocation, metastoredb)
+    val catalog = CarbonEnv.getInstance(spark).carbonMetastore
+    sampleRelation = catalog.lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+      "sample")(spark).asInstanceOf[CarbonRelation]
+    complexRelation = catalog.lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+      "complextypes")(spark).asInstanceOf[CarbonRelation]
+  }
+
+  test("Support generate global dictionary from all dictionary files") {
+    val header = "id,name,city,age"
+    val carbonLoadModel = buildCarbonLoadModel(sampleRelation, null, header, sampleAllDictionaryFile)
+    GlobalDictionaryUtil
+      .generateGlobalDictionary(sqlContext,
+        carbonLoadModel,
+        sampleRelation.tableMeta.storePath)
+
+    DictionaryTestCaseUtil.
+      checkDictionary(sampleRelation, "city", "shenzhen")
+  }
+
+  test("Support generate global dictionary from all dictionary files for complex type") {
+    val header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC,locationinfo,proddate,gamePointId,contractNumber"
+    val carbonLoadModel = buildCarbonLoadModel(complexRelation, null, header, complexAllDictionaryFile)
+    GlobalDictionaryUtil
+      .generateGlobalDictionary(sqlContext,
+      carbonLoadModel,
+      complexRelation.tableMeta.storePath)
+
+    DictionaryTestCaseUtil.
+      checkDictionary(complexRelation, "channelsId", "1650")
+  }
+  
+  override def afterAll {
+    sql("drop table sample")
+    sql("drop table complextypes")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
new file mode 100644
index 0000000..62b0aff
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.carbondata.spark.util
+
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.test.TestQueryExecutor
+
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.spark.load.CarbonLoaderUtil
+
+/**
+ * Utility for global dictionary test cases
+ */
+object DictionaryTestCaseUtil {
+
+  /**
+   *  check whether the dictionary of specified column generated
+   * @param relation  carbon table relation
+   * @param columnName  name of specified column
+   * @param value  a value of column
+   */
+  def checkDictionary(relation: CarbonRelation, columnName: String, value: String) {
+    val table = relation.tableMeta.carbonTable
+    val dimension = table.getDimensionByName(table.getFactTableName, columnName)
+    val tableIdentifier = new CarbonTableIdentifier(table.getDatabaseName, table.getFactTableName,
"uniqueid")
+    val columnIdentifier = new DictionaryColumnUniqueIdentifier(tableIdentifier,
+      dimension.getColumnIdentifier, dimension.getDataType
+    )
+    val dict = CarbonLoaderUtil.getDictionary(columnIdentifier, TestQueryExecutor.storeLocation)
+    assert(dict.getSurrogateKey(value) != CarbonCommonConstants.INVALID_SURROGATE_KEY)
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/083a28a8/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
new file mode 100644
index 0000000..10f99b7
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -0,0 +1,284 @@
+/*
+  * 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.carbondata.spark.util
+
+import org.apache.spark.sql.common.util.QueryTest
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.constants.TableOptionConstant
+import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+/**
+ * test case for external column dictionary generation
+ * also support complicated type
+ */
+class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  var extComplexRelation: CarbonRelation = _
+  var verticalDelimiteRelation: CarbonRelation = _
+  var loadSqlRelation: CarbonRelation = _
+  var filePath: String = _
+  var pwd: String = _
+  var complexFilePath1: String = _
+  var complexFilePath2: String = _
+  var extColDictFilePath1: String = _
+  var extColDictFilePath2: String = _
+  var extColDictFilePath3: String = _
+  var header: String = _
+  var header2: String = _
+
+  def buildTestData() = {
+
+    filePath = s"${ resourcesPath }/sample.csv"
+    complexFilePath1 = s"${ resourcesPath }/complexdata2.csv"
+    complexFilePath2 = s"${ resourcesPath }/verticalDelimitedData.csv"
+    extColDictFilePath1 = s"deviceInformationId:${ resourcesPath }/deviceInformationId.csv,"
+
+                          s"mobile.imei:${ resourcesPath }/mobileimei.csv," +
+                          s"mac:${ resourcesPath }/mac.csv," +
+                          s"locationInfo.ActiveCountry:${ resourcesPath
+                          }/locationInfoActiveCountry.csv"
+    extColDictFilePath2 = s"deviceInformationId:${ resourcesPath }/deviceInformationId2.csv"
+    extColDictFilePath3 = s"channelsId:${ resourcesPath }/channelsId.csv"
+    header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC," +
+             "locationinfo,proddate,gamePointId,contractNumber"
+    header2 = "deviceInformationId,channelsId,contractNumber"
+  }
+
+  def buildTable() = {
+    try {
+      sql(
+        """CREATE TABLE extComplextypes (deviceInformationId int,
+     channelsId string, ROMSize string, purchasedate string,
+     mobile struct<imei:string, imsi:string>, MAC array<string>,
+     locationinfo array<struct<ActiveAreaId:int, ActiveCountry:string,
+     ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>,
+     proddate struct<productionDate:string,activeDeactivedate:array<string>>,
+     gamePointId double,contractNumber double)
+     STORED BY 'org.apache.carbondata.format'
+     TBLPROPERTIES('DICTIONARY_INCLUDE' = 'deviceInformationId')
+        """)
+    } catch {
+      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
+    }
+
+    try {
+      sql(
+        """CREATE TABLE verticalDelimitedTable (deviceInformationId int,
+     channelsId string,contractNumber double)
+     STORED BY 'org.apache.carbondata.format'
+     TBLPROPERTIES('DICTIONARY_INCLUDE' = 'deviceInformationId')
+        """)
+    } catch {
+      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
+    }
+
+    try {
+      sql(
+        """CREATE TABLE loadSqlTest (deviceInformationId int,
+     channelsId string, ROMSize string, purchasedate string,
+     mobile struct<imei:string, imsi:string>, MAC array<string>,
+     locationinfo array<struct<ActiveAreaId:int, ActiveCountry:string,
+     ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>,
+     proddate struct<productionDate:string,activeDeactivedate:array<string>>,
+     gamePointId double,contractNumber double)
+     STORED BY 'org.apache.carbondata.format'
+     TBLPROPERTIES('DICTIONARY_INCLUDE' = 'deviceInformationId')
+        """)
+    } catch {
+      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
+    }
+  }
+
+  def buildRelation() = {
+    val warehouse = s"$resourcesPath/target/warehouse"
+    val storeLocation = s"$resourcesPath/target/store"
+    val metastoredb = s"$resourcesPath/target"
+    CarbonProperties.getInstance()
+      .addProperty("carbon.custom.distribution", "true")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,"FORCE")
+    import org.apache.spark.sql.CarbonSession._
+
+    val spark = SparkSession
+      .builder()
+      .master("local")
+      .appName("CarbonSessionExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .config("spark.network.timeout", "600s")
+      .config("spark.executor.heartbeatInterval", "600s")
+      .config("carbon.enable.vector.reader","false")
+      .getOrCreateCarbonSession(storeLocation, metastoredb)
+    val catalog = CarbonEnv.getInstance(spark).carbonMetastore
+    extComplexRelation = catalog
+      .lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+        "extComplextypes")(spark)
+      .asInstanceOf[CarbonRelation]
+    verticalDelimiteRelation = catalog
+      .lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+        "verticalDelimitedTable")(spark)
+      .asInstanceOf[CarbonRelation]
+    loadSqlRelation = catalog.lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+      "loadSqlTest")(spark)
+      .asInstanceOf[CarbonRelation]
+  }
+
+  def buildCarbonLoadModel(relation: CarbonRelation,
+      filePath: String,
+      header: String,
+      extColFilePath: String,
+      csvDelimiter: String = ","): CarbonLoadModel = {
+    val carbonLoadModel = new CarbonLoadModel
+    carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
+    carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    val table = relation.tableMeta.carbonTable
+    val carbonSchema = new CarbonDataLoadSchema(table)
+    carbonLoadModel.setDatabaseName(table.getDatabaseName)
+    carbonLoadModel.setTableName(table.getFactTableName)
+    carbonLoadModel.setCarbonDataLoadSchema(carbonSchema)
+    carbonLoadModel.setFactFilePath(filePath)
+    carbonLoadModel.setCsvHeader(header)
+    carbonLoadModel.setCsvDelimiter(csvDelimiter)
+    carbonLoadModel.setComplexDelimiterLevel1("\\$")
+    carbonLoadModel.setComplexDelimiterLevel2("\\:")
+    carbonLoadModel.setColDictFilePath(extColFilePath)
+    carbonLoadModel.setQuoteChar("\"");
+    carbonLoadModel.setSerializationNullFormat(
+      TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + ",\\N")
+    carbonLoadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+    carbonLoadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.CARBON_DATE_FORMAT,
+      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+    carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+    carbonLoadModel.setMaxColumns("100")
+    carbonLoadModel
+  }
+
+  override def beforeAll {
+    buildTestData
+    buildTable
+    buildRelation
+  }
+
+  test("Generate global dictionary from external column file") {
+    // load the first time
+    var carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
+      header, extColDictFilePath1)
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
+      extComplexRelation.tableMeta.storePath)
+    // check whether the dictionary is generated
+    DictionaryTestCaseUtil.checkDictionary(
+      extComplexRelation, "deviceInformationId", "10086")
+
+    // load the second time
+    carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
+      header, extColDictFilePath2)
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
+      extComplexRelation.tableMeta.storePath)
+    // check the old dictionary and whether the new distinct value is generated
+    DictionaryTestCaseUtil.checkDictionary(
+      extComplexRelation, "deviceInformationId", "10086")
+    DictionaryTestCaseUtil.checkDictionary(
+      extComplexRelation, "deviceInformationId", "10011")
+  }
+
+  test("When csv delimiter is not comma") {
+    //  when csv delimiter is comma
+    var carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
+      header, extColDictFilePath3)
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
+      extComplexRelation.tableMeta.storePath)
+    // check whether the dictionary is generated
+    DictionaryTestCaseUtil.checkDictionary(
+      extComplexRelation, "channelsId", "1421|")
+
+    //  when csv delimiter is not comma
+    carbonLoadModel = buildCarbonLoadModel(verticalDelimiteRelation, complexFilePath2,
+      header2, extColDictFilePath3, "|")
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
+      verticalDelimiteRelation.tableMeta.storePath)
+    // check whether the dictionary is generated
+    DictionaryTestCaseUtil.checkDictionary(
+      verticalDelimiteRelation, "channelsId", "1431,")
+  }
+
+  test("LOAD DML with COLUMNDICT option") {
+    try {
+      sql(
+        s"""
+      LOAD DATA LOCAL INPATH "$complexFilePath1" INTO TABLE loadSqlTest
+      OPTIONS('FILEHEADER'='$header', 'COLUMNDICT'='$extColDictFilePath1')
+        """)
+    } catch {
+      case ex: Exception =>
+        LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
+        assert(false)
+    }
+    DictionaryTestCaseUtil.checkDictionary(
+      loadSqlRelation, "deviceInformationId", "10086")
+  }
+
+  test("COLUMNDICT and ALL_DICTIONARY_PATH can not be used together") {
+    try {
+      sql(
+        s"""
+        LOAD DATA LOCAL INPATH "$complexFilePath1" INTO TABLE loadSqlTest
+        OPTIONS('COLUMNDICT'='$extColDictFilePath1',"ALL_DICTIONARY_PATH"='$extColDictFilePath1')
+        """)
+      assert(false)
+    } catch {
+      case ex: MalformedCarbonCommandException =>
+        assertResult(ex.getMessage)(
+          "Error: COLUMNDICT and ALL_DICTIONARY_PATH can not be used together " +
+          "in options")
+      case _: Throwable => assert(false)
+    }
+  }
+
+  test("Measure can not use COLUMNDICT") {
+    try {
+      sql(
+        s"""
+      LOAD DATA LOCAL INPATH "$complexFilePath1" INTO TABLE loadSqlTest
+      OPTIONS('FILEHEADER'='$header', 'COLUMNDICT'='gamePointId:$filePath')
+      """)
+      assert(false)
+    } catch {
+      case ex: DataLoadingException =>
+        assertResult(ex.getMessage)(
+          "Column gamePointId is not a key column. Only key column can be part " +
+          "of dictionary and used in COLUMNDICT option.")
+      case _: Throwable => assert(false)
+    }
+  }
+
+  override def afterAll: Unit = {
+    sql("DROP TABLE extComplextypes")
+    sql("DROP TABLE verticalDelimitedTable")
+    sql("DROP TABLE loadSqlTest")
+  }
+}


Mime
View raw message